From b18aeaa4a9347701ffffa547362da0a1f2f5215e Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Tue, 27 May 2025 22:39:19 +0800 Subject: [PATCH 01/74] feat: support inability to yeild cpu for loop when it's not using Tokio MPSC (RecordBatchReceiverStream) --- .../physical-plan/src/aggregates/mod.rs | 6 ++- .../src/aggregates/no_grouping.rs | 50 +++++++++++++++++++ parquet-testing | 2 +- 3 files changed, 55 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2062e2208b40..55274847eb70 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -47,6 +47,7 @@ use datafusion_physical_expr::{ PhysicalSortRequirement, }; +use crate::aggregates::no_grouping::YieldStream; use datafusion_physical_expr_common::physical_expr::fmt_sql; use itertools::Itertools; @@ -983,8 +984,9 @@ impl ExecutionPlan for AggregateExec { partition: usize, context: Arc, ) -> Result { - self.execute_typed(partition, context) - .map(|stream| stream.into()) + let raw_stream = self.execute_typed(partition, context)?.into(); + let wrapped = Box::pin(YieldStream::new(raw_stream)); + Ok(wrapped) } fn metrics(&self) -> Option { diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 9474a5f88c92..09e5068906dd 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -76,6 +76,7 @@ impl AggregateStream { let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let input = agg.input.execute(partition, Arc::clone(&context))?; + let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; let filter_expressions = match agg.mode { @@ -170,6 +171,55 @@ impl AggregateStream { } } +/// A stream that yields batches of data, yielding control back to the executor +pub struct YieldStream { + inner: SendableRecordBatchStream, + batches_processed: usize, +} + +impl YieldStream { + pub fn new(inner: SendableRecordBatchStream) -> Self { + Self { + inner, + batches_processed: 0, + } + } +} + +// Stream> to poll_next_unpin +impl Stream for YieldStream { + type Item = Result; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + const YIELD_BATCHES: usize = 64; + let this = &mut *self; + + match this.inner.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + this.batches_processed += 1; + if this.batches_processed >= YIELD_BATCHES { + this.batches_processed = 0; + cx.waker().wake_by_ref(); + Poll::Pending + } else { + Poll::Ready(Some(Ok(batch))) + } + } + other => other, + } + } +} + +// RecordBatchStream schema() +impl RecordBatchStream for YieldStream { + fn schema(&self) -> Arc { + self.inner.schema() + } +} + impl Stream for AggregateStream { type Item = Result; diff --git a/parquet-testing b/parquet-testing index 6e851ddd768d..107b36603e05 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit 6e851ddd768d6af741c7b15dc594874399fc3cff +Subproject commit 107b36603e051aee26bd93e04b871034f6c756c0 From 2a965aa01bc96d738f62b87ad14bfed6979c1276 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 28 May 2025 18:06:44 +0800 Subject: [PATCH 02/74] Fix fuzz test --- datafusion/physical-plan/src/aggregates/mod.rs | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 55274847eb70..d4bcbec6c5d2 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -984,9 +984,18 @@ impl ExecutionPlan for AggregateExec { partition: usize, context: Arc, ) -> Result { - let raw_stream = self.execute_typed(partition, context)?.into(); - let wrapped = Box::pin(YieldStream::new(raw_stream)); - Ok(wrapped) + // Only wrap *no‐grouping* aggregates in our YieldStream + // (grouped aggregates tend to produce small streams + // and can rely on Tokio's own task‐yielding) + let typed = self.execute_typed(partition, context)?; + if self.group_expr().is_empty() { + // no GROUP BY: inject our yield wrapper + let raw_stream = typed.into(); // SendableRecordBatchStream + Ok(Box::pin(YieldStream::new(raw_stream))) + } else { + // has GROUP BY: just hand back the raw stream + Ok(typed.into()) + } } fn metrics(&self) -> Option { From 67ca44b4c69c93e4b4e71e9610a3188ad124cd20 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 28 May 2025 18:17:32 +0800 Subject: [PATCH 03/74] polish code --- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/aggregates/no_grouping.rs | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index d4bcbec6c5d2..b31b44b87744 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -984,7 +984,7 @@ impl ExecutionPlan for AggregateExec { partition: usize, context: Arc, ) -> Result { - // Only wrap *no‐grouping* aggregates in our YieldStream + // Only wrap no‐grouping aggregates in our YieldStream // (grouped aggregates tend to produce small streams // and can rely on Tokio's own task‐yielding) let typed = self.execute_typed(partition, context)?; diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 09e5068906dd..5c1de060018c 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -76,7 +76,6 @@ impl AggregateStream { let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let input = agg.input.execute(partition, Arc::clone(&context))?; - let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; let filter_expressions = match agg.mode { From 82a179d3248aa5ebe245b2ad1b5d23d0dcf5230c Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 28 May 2025 18:29:56 +0800 Subject: [PATCH 04/74] add comments --- datafusion/physical-plan/src/aggregates/mod.rs | 15 ++------------- .../physical-plan/src/aggregates/no_grouping.rs | 5 +++++ 2 files changed, 7 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b31b44b87744..2062e2208b40 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -47,7 +47,6 @@ use datafusion_physical_expr::{ PhysicalSortRequirement, }; -use crate::aggregates::no_grouping::YieldStream; use datafusion_physical_expr_common::physical_expr::fmt_sql; use itertools::Itertools; @@ -984,18 +983,8 @@ impl ExecutionPlan for AggregateExec { partition: usize, context: Arc, ) -> Result { - // Only wrap no‐grouping aggregates in our YieldStream - // (grouped aggregates tend to produce small streams - // and can rely on Tokio's own task‐yielding) - let typed = self.execute_typed(partition, context)?; - if self.group_expr().is_empty() { - // no GROUP BY: inject our yield wrapper - let raw_stream = typed.into(); // SendableRecordBatchStream - Ok(Box::pin(YieldStream::new(raw_stream))) - } else { - // has GROUP BY: just hand back the raw stream - Ok(typed.into()) - } + self.execute_typed(partition, context) + .map(|stream| stream.into()) } fn metrics(&self) -> Option { diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 5c1de060018c..5f81e7b9dccb 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -77,6 +77,11 @@ impl AggregateStream { let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let input = agg.input.execute(partition, Arc::clone(&context))?; + // Only wrap no‐grouping aggregates in our YieldStream + // (grouped aggregates tend to produce small streams + // and can rely on Tokio's own task‐yielding) + let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; + let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; let filter_expressions = match agg.mode { AggregateMode::Partial From da3c2d57b71192f8d942dbdfc3491838b958d2d5 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 28 May 2025 19:22:38 +0800 Subject: [PATCH 05/74] fix corner case when huge data --- datafusion/physical-plan/src/aggregates/no_grouping.rs | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 5f81e7b9dccb..78cd42c04360 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -179,6 +179,7 @@ impl AggregateStream { pub struct YieldStream { inner: SendableRecordBatchStream, batches_processed: usize, + buffer: Option>, } impl YieldStream { @@ -186,6 +187,7 @@ impl YieldStream { Self { inner, batches_processed: 0, + buffer: None, } } } @@ -201,11 +203,19 @@ impl Stream for YieldStream { const YIELD_BATCHES: usize = 64; let this = &mut *self; + if let Some(batch) = this.buffer.take() { + return Poll::Ready(Some(batch)); + } + match this.inner.poll_next_unpin(cx) { Poll::Ready(Some(Ok(batch))) => { this.batches_processed += 1; if this.batches_processed >= YIELD_BATCHES { this.batches_processed = 0; + // We need to buffer the batch when we return Poll::Pending, + // so that we can return it on the next poll. + // Otherwise, the next poll will miss the batch and return None. + this.buffer = Some(Ok(batch)); cx.waker().wake_by_ref(); Poll::Pending } else { From 6cf3bf0fbc474a33030184fa7f0f6c013db7c7b7 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 28 May 2025 20:06:24 +0800 Subject: [PATCH 06/74] Also add grouping case --- datafusion/physical-plan/src/aggregates/no_grouping.rs | 4 +--- datafusion/physical-plan/src/aggregates/row_hash.rs | 5 +++++ 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 78cd42c04360..4a90618a0d01 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -77,9 +77,7 @@ impl AggregateStream { let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let input = agg.input.execute(partition, Arc::clone(&context))?; - // Only wrap no‐grouping aggregates in our YieldStream - // (grouped aggregates tend to produce small streams - // and can rely on Tokio's own task‐yielding) + // Wrap no‐grouping aggregates in our YieldStream let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 62f541443068..7c79033bc9e7 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -49,6 +49,7 @@ use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; use super::order::GroupOrdering; use super::AggregateExec; +use crate::aggregates::no_grouping::YieldStream; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::ready; @@ -448,6 +449,10 @@ impl GroupedHashAggregateStream { let batch_size = context.session_config().batch_size(); let input = agg.input.execute(partition, Arc::clone(&context))?; + + // Wrap grouping aggregates in our YieldStream + let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; + let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let timer = baseline_metrics.elapsed_compute().timer(); From 325199083c38f51f0f2296128a3848cc0bbb7c8b Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 30 May 2025 11:40:01 +0800 Subject: [PATCH 07/74] Address comments --- Cargo.lock | 4 + datafusion/common-runtime/Cargo.toml | 4 + datafusion/common-runtime/src/common.rs | 147 +++++++++++++++++- .../src/aggregates/no_grouping.rs | 65 +------- .../physical-plan/src/aggregates/row_hash.rs | 4 +- 5 files changed, 159 insertions(+), 65 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7641caab928a..b912353c6240 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2005,6 +2005,10 @@ dependencies = [ name = "datafusion-common-runtime" version = "47.0.0" dependencies = [ + "arrow", + "arrow-schema", + "datafusion-common", + "datafusion-execution", "futures", "log", "tokio", diff --git a/datafusion/common-runtime/Cargo.toml b/datafusion/common-runtime/Cargo.toml index 7ddc021e640c..b0c433e52a18 100644 --- a/datafusion/common-runtime/Cargo.toml +++ b/datafusion/common-runtime/Cargo.toml @@ -38,6 +38,10 @@ workspace = true name = "datafusion_common_runtime" [dependencies] +arrow = { workspace = true } +arrow-schema = { workspace = true } +datafusion-execution = { workspace = true } +datafusion-common = { workspace = true, default-features = true } futures = { workspace = true } log = { workspace = true } tokio = { workspace = true } diff --git a/datafusion/common-runtime/src/common.rs b/datafusion/common-runtime/src/common.rs index e7aba1d455ee..9bbec88f7103 100644 --- a/datafusion/common-runtime/src/common.rs +++ b/datafusion/common-runtime/src/common.rs @@ -15,14 +15,19 @@ // specific language governing permissions and limitations // under the License. +use futures::{Stream, StreamExt}; +use std::sync::Arc; use std::{ future::Future, pin::Pin, task::{Context, Poll}, }; - use tokio::task::{JoinError, JoinHandle}; +use arrow::record_batch::RecordBatch; +use datafusion_common::Result; +use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; + use crate::trace_utils::{trace_block, trace_future}; /// Helper that provides a simple API to spawn a single task and join it. @@ -98,6 +103,71 @@ impl Drop for SpawnedTask { } } +/// Number of batches to yield before voluntarily returning Pending. +/// This allows long-running operators to periodically yield control +/// back to the executor (e.g., to handle cancellation). +const YIELD_BATCHES: usize = 64; + +/// A stream that yields batches of data, yielding control back to the executor every `YIELD_BATCHES` batches +/// +/// This can be useful to allow operators that might not yield to check for cancellation +pub struct YieldStream { + inner: SendableRecordBatchStream, + batches_processed: usize, + buffer: Option>, +} + +impl YieldStream { + pub fn new(inner: SendableRecordBatchStream) -> Self { + Self { + inner, + batches_processed: 0, + buffer: None, + } + } +} + +// Stream> to poll_next_unpin +impl Stream for YieldStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let this = &mut *self; + + if let Some(batch) = this.buffer.take() { + return Poll::Ready(Some(batch)); + } + + match this.inner.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + this.batches_processed += 1; + if this.batches_processed >= YIELD_BATCHES { + this.batches_processed = 0; + // We need to buffer the batch when we return Poll::Pending, + // so that we can return it on the next poll. + // Otherwise, the next poll will miss the batch and return None. + this.buffer = Some(Ok(batch)); + cx.waker().wake_by_ref(); + Poll::Pending + } else { + Poll::Ready(Some(Ok(batch))) + } + } + other => other, + } + } +} + +// RecordBatchStream schema() +impl RecordBatchStream for YieldStream { + fn schema(&self) -> Arc { + self.inner.schema() + } +} + #[cfg(test)] mod tests { use super::*; @@ -106,6 +176,17 @@ mod tests { use tokio::{runtime::Runtime, sync::oneshot}; + use arrow::datatypes::SchemaRef; + use arrow_schema::Schema; + use datafusion_common::Result; + use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; + use futures::{stream, Stream, StreamExt, TryStreamExt}; + use std::{ + pin::Pin, + sync::Arc, + task::{Context, Poll}, + }; + #[tokio::test] async fn runtime_shutdown() { let rt = Runtime::new().unwrap(); @@ -173,4 +254,68 @@ mod tests { // The sender was dropped so we receive `None`. assert!(receiver.recv().await.is_none()); } + + /// A tiny adapter that turns any `Stream>` + /// into a `RecordBatchStream` by carrying along a schema. + struct EmptyBatchStream { + inner: Pin> + Send>>, + schema: SchemaRef, + } + + impl Stream for EmptyBatchStream { + type Item = Result; + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + Pin::new(&mut self.inner).poll_next(cx) + } + } + + impl RecordBatchStream for EmptyBatchStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } + } + + /// Helper: construct a SendableRecordBatchStream containing `n` empty batches + fn make_empty_batches(n: usize) -> SendableRecordBatchStream { + let schema: SchemaRef = Arc::new(Schema::empty()); + let schema_for_stream = Arc::clone(&schema); + + let s = + stream::iter((0..n).map(move |_| { + Ok(RecordBatch::new_empty(Arc::clone(&schema_for_stream))) + })) + .boxed(); + + Box::pin(EmptyBatchStream { inner: s, schema }) + } + + #[tokio::test] + async fn yield_less_than_threshold() -> Result<()> { + let count = YIELD_BATCHES - 10; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; + assert_eq!(out.len(), count); + Ok(()) + } + + #[tokio::test] + async fn yield_equal_to_threshold() -> Result<()> { + let count = YIELD_BATCHES; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; + assert_eq!(out.len(), count); + Ok(()) + } + + #[tokio::test] + async fn yield_more_than_threshold() -> Result<()> { + let count = YIELD_BATCHES + 20; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; + assert_eq!(out.len(), count); + Ok(()) + } } diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 4a90618a0d01..767d2d42235d 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -33,12 +33,12 @@ use std::borrow::Cow; use std::sync::Arc; use std::task::{Context, Poll}; +use super::AggregateExec; use crate::filter::batch_filter; +use datafusion_common_runtime::common::YieldStream; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use futures::stream::{Stream, StreamExt}; -use super::AggregateExec; - /// stream struct for aggregation without grouping columns pub(crate) struct AggregateStream { stream: BoxStream<'static, Result>, @@ -77,7 +77,7 @@ impl AggregateStream { let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let input = agg.input.execute(partition, Arc::clone(&context))?; - // Wrap no‐grouping aggregates in our YieldStream + // Yield control back to tokio after a certain number of batches so it can check for cancellation. let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; @@ -173,65 +173,6 @@ impl AggregateStream { } } -/// A stream that yields batches of data, yielding control back to the executor -pub struct YieldStream { - inner: SendableRecordBatchStream, - batches_processed: usize, - buffer: Option>, -} - -impl YieldStream { - pub fn new(inner: SendableRecordBatchStream) -> Self { - Self { - inner, - batches_processed: 0, - buffer: None, - } - } -} - -// Stream> to poll_next_unpin -impl Stream for YieldStream { - type Item = Result; - - fn poll_next( - mut self: std::pin::Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - const YIELD_BATCHES: usize = 64; - let this = &mut *self; - - if let Some(batch) = this.buffer.take() { - return Poll::Ready(Some(batch)); - } - - match this.inner.poll_next_unpin(cx) { - Poll::Ready(Some(Ok(batch))) => { - this.batches_processed += 1; - if this.batches_processed >= YIELD_BATCHES { - this.batches_processed = 0; - // We need to buffer the batch when we return Poll::Pending, - // so that we can return it on the next poll. - // Otherwise, the next poll will miss the batch and return None. - this.buffer = Some(Ok(batch)); - cx.waker().wake_by_ref(); - Poll::Pending - } else { - Poll::Ready(Some(Ok(batch))) - } - } - other => other, - } - } -} - -// RecordBatchStream schema() -impl RecordBatchStream for YieldStream { - fn schema(&self) -> Arc { - self.inner.schema() - } -} - impl Stream for AggregateStream { type Item = Result; diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 7c79033bc9e7..41ddae2e7b88 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -49,7 +49,7 @@ use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; use super::order::GroupOrdering; use super::AggregateExec; -use crate::aggregates::no_grouping::YieldStream; +use datafusion_common_runtime::common::YieldStream; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::ready; @@ -450,7 +450,7 @@ impl GroupedHashAggregateStream { let batch_size = context.session_config().batch_size(); let input = agg.input.execute(partition, Arc::clone(&context))?; - // Wrap grouping aggregates in our YieldStream + // Yield control back to tokio after a certain number of batches so it can check for cancellation. let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); From 311849d9cfaaed51717a4a99cef518bc33556b55 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 30 May 2025 12:17:56 +0800 Subject: [PATCH 08/74] fmt --- datafusion/common-runtime/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/common-runtime/Cargo.toml b/datafusion/common-runtime/Cargo.toml index b0c433e52a18..cb07591a6a02 100644 --- a/datafusion/common-runtime/Cargo.toml +++ b/datafusion/common-runtime/Cargo.toml @@ -40,8 +40,8 @@ name = "datafusion_common_runtime" [dependencies] arrow = { workspace = true } arrow-schema = { workspace = true } -datafusion-execution = { workspace = true } datafusion-common = { workspace = true, default-features = true } +datafusion-execution = { workspace = true } futures = { workspace = true } log = { workspace = true } tokio = { workspace = true } From 2a2ead95c1a52818ae66374951e5ad23ab2ed39e Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 30 May 2025 06:04:45 -0400 Subject: [PATCH 09/74] Move YieldStream into physical-plan crate --- Cargo.lock | 4 - datafusion/common-runtime/Cargo.toml | 4 - datafusion/common-runtime/src/common.rs | 147 +----------------- .../src/aggregates/no_grouping.rs | 3 +- .../physical-plan/src/aggregates/row_hash.rs | 3 +- datafusion/physical-plan/src/stream.rs | 130 ++++++++++++++++ parquet-testing | 2 +- 7 files changed, 135 insertions(+), 158 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e448d78cb48b..b4ad326358f8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2005,10 +2005,6 @@ dependencies = [ name = "datafusion-common-runtime" version = "47.0.0" dependencies = [ - "arrow", - "arrow-schema", - "datafusion-common", - "datafusion-execution", "futures", "log", "tokio", diff --git a/datafusion/common-runtime/Cargo.toml b/datafusion/common-runtime/Cargo.toml index cb07591a6a02..7ddc021e640c 100644 --- a/datafusion/common-runtime/Cargo.toml +++ b/datafusion/common-runtime/Cargo.toml @@ -38,10 +38,6 @@ workspace = true name = "datafusion_common_runtime" [dependencies] -arrow = { workspace = true } -arrow-schema = { workspace = true } -datafusion-common = { workspace = true, default-features = true } -datafusion-execution = { workspace = true } futures = { workspace = true } log = { workspace = true } tokio = { workspace = true } diff --git a/datafusion/common-runtime/src/common.rs b/datafusion/common-runtime/src/common.rs index 9bbec88f7103..e7aba1d455ee 100644 --- a/datafusion/common-runtime/src/common.rs +++ b/datafusion/common-runtime/src/common.rs @@ -15,18 +15,13 @@ // specific language governing permissions and limitations // under the License. -use futures::{Stream, StreamExt}; -use std::sync::Arc; use std::{ future::Future, pin::Pin, task::{Context, Poll}, }; -use tokio::task::{JoinError, JoinHandle}; -use arrow::record_batch::RecordBatch; -use datafusion_common::Result; -use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; +use tokio::task::{JoinError, JoinHandle}; use crate::trace_utils::{trace_block, trace_future}; @@ -103,71 +98,6 @@ impl Drop for SpawnedTask { } } -/// Number of batches to yield before voluntarily returning Pending. -/// This allows long-running operators to periodically yield control -/// back to the executor (e.g., to handle cancellation). -const YIELD_BATCHES: usize = 64; - -/// A stream that yields batches of data, yielding control back to the executor every `YIELD_BATCHES` batches -/// -/// This can be useful to allow operators that might not yield to check for cancellation -pub struct YieldStream { - inner: SendableRecordBatchStream, - batches_processed: usize, - buffer: Option>, -} - -impl YieldStream { - pub fn new(inner: SendableRecordBatchStream) -> Self { - Self { - inner, - batches_processed: 0, - buffer: None, - } - } -} - -// Stream> to poll_next_unpin -impl Stream for YieldStream { - type Item = Result; - - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - let this = &mut *self; - - if let Some(batch) = this.buffer.take() { - return Poll::Ready(Some(batch)); - } - - match this.inner.poll_next_unpin(cx) { - Poll::Ready(Some(Ok(batch))) => { - this.batches_processed += 1; - if this.batches_processed >= YIELD_BATCHES { - this.batches_processed = 0; - // We need to buffer the batch when we return Poll::Pending, - // so that we can return it on the next poll. - // Otherwise, the next poll will miss the batch and return None. - this.buffer = Some(Ok(batch)); - cx.waker().wake_by_ref(); - Poll::Pending - } else { - Poll::Ready(Some(Ok(batch))) - } - } - other => other, - } - } -} - -// RecordBatchStream schema() -impl RecordBatchStream for YieldStream { - fn schema(&self) -> Arc { - self.inner.schema() - } -} - #[cfg(test)] mod tests { use super::*; @@ -176,17 +106,6 @@ mod tests { use tokio::{runtime::Runtime, sync::oneshot}; - use arrow::datatypes::SchemaRef; - use arrow_schema::Schema; - use datafusion_common::Result; - use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; - use futures::{stream, Stream, StreamExt, TryStreamExt}; - use std::{ - pin::Pin, - sync::Arc, - task::{Context, Poll}, - }; - #[tokio::test] async fn runtime_shutdown() { let rt = Runtime::new().unwrap(); @@ -254,68 +173,4 @@ mod tests { // The sender was dropped so we receive `None`. assert!(receiver.recv().await.is_none()); } - - /// A tiny adapter that turns any `Stream>` - /// into a `RecordBatchStream` by carrying along a schema. - struct EmptyBatchStream { - inner: Pin> + Send>>, - schema: SchemaRef, - } - - impl Stream for EmptyBatchStream { - type Item = Result; - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - Pin::new(&mut self.inner).poll_next(cx) - } - } - - impl RecordBatchStream for EmptyBatchStream { - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } - } - - /// Helper: construct a SendableRecordBatchStream containing `n` empty batches - fn make_empty_batches(n: usize) -> SendableRecordBatchStream { - let schema: SchemaRef = Arc::new(Schema::empty()); - let schema_for_stream = Arc::clone(&schema); - - let s = - stream::iter((0..n).map(move |_| { - Ok(RecordBatch::new_empty(Arc::clone(&schema_for_stream))) - })) - .boxed(); - - Box::pin(EmptyBatchStream { inner: s, schema }) - } - - #[tokio::test] - async fn yield_less_than_threshold() -> Result<()> { - let count = YIELD_BATCHES - 10; - let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; - assert_eq!(out.len(), count); - Ok(()) - } - - #[tokio::test] - async fn yield_equal_to_threshold() -> Result<()> { - let count = YIELD_BATCHES; - let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; - assert_eq!(out.len(), count); - Ok(()) - } - - #[tokio::test] - async fn yield_more_than_threshold() -> Result<()> { - let count = YIELD_BATCHES + 20; - let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; - assert_eq!(out.len(), count); - Ok(()) - } } diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 767d2d42235d..0651856a1463 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -35,7 +35,8 @@ use std::task::{Context, Poll}; use super::AggregateExec; use crate::filter::batch_filter; -use datafusion_common_runtime::common::YieldStream; + +use crate::stream::YieldStream; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use futures::stream::{Stream, StreamExt}; diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 41ddae2e7b88..c10aecb4cc3a 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -31,7 +31,7 @@ use crate::metrics::{BaselineMetrics, MetricBuilder, RecordOutput}; use crate::sorts::sort::sort_batch; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::spill_manager::SpillManager; -use crate::stream::RecordBatchStreamAdapter; +use crate::stream::{RecordBatchStreamAdapter, YieldStream}; use crate::{aggregates, metrics, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; @@ -49,7 +49,6 @@ use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; use super::order::GroupOrdering; use super::AggregateExec; -use datafusion_common_runtime::common::YieldStream; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::ready; diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index 338ac7d048a3..57a829d3e69d 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -522,6 +522,71 @@ impl Stream for ObservedStream { } } +/// Number of batches to yield before voluntarily returning Pending. +/// This allows long-running operators to periodically yield control +/// back to the executor (e.g., to handle cancellation). +const YIELD_BATCHES: usize = 64; + +/// A stream that yields batches of data, yielding control back to the executor every `YIELD_BATCHES` batches +/// +/// This can be useful to allow operators that might not yield to check for cancellation +pub struct YieldStream { + inner: SendableRecordBatchStream, + batches_processed: usize, + buffer: Option>, +} + +impl YieldStream { + pub fn new(inner: SendableRecordBatchStream) -> Self { + Self { + inner, + batches_processed: 0, + buffer: None, + } + } +} + +// Stream> to poll_next_unpin +impl Stream for YieldStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let this = &mut *self; + + if let Some(batch) = this.buffer.take() { + return Poll::Ready(Some(batch)); + } + + match this.inner.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + this.batches_processed += 1; + if this.batches_processed >= YIELD_BATCHES { + this.batches_processed = 0; + // We need to buffer the batch when we return Poll::Pending, + // so that we can return it on the next poll. + // Otherwise, the next poll will miss the batch and return None. + this.buffer = Some(Ok(batch)); + cx.waker().wake_by_ref(); + Poll::Pending + } else { + Poll::Ready(Some(Ok(batch))) + } + } + other => other, + } + } +} + +// RecordBatchStream schema() +impl RecordBatchStream for YieldStream { + fn schema(&self) -> Arc { + self.inner.schema() + } +} + #[cfg(test)] mod test { use super::*; @@ -531,6 +596,7 @@ mod test { use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::exec_err; + use futures::{stream, TryStreamExt}; fn schema() -> SchemaRef { Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)])) @@ -649,4 +715,68 @@ mod test { ); } } + + /// A tiny adapter that turns any `Stream>` + /// into a `RecordBatchStream` by carrying along a schema. + struct EmptyBatchStream { + inner: Pin> + Send>>, + schema: SchemaRef, + } + + impl Stream for EmptyBatchStream { + type Item = Result; + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + Pin::new(&mut self.inner).poll_next(cx) + } + } + + impl RecordBatchStream for EmptyBatchStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } + } + + /// Helper: construct a SendableRecordBatchStream containing `n` empty batches + fn make_empty_batches(n: usize) -> SendableRecordBatchStream { + let schema: SchemaRef = Arc::new(Schema::empty()); + let schema_for_stream = Arc::clone(&schema); + + let s = + stream::iter((0..n).map(move |_| { + Ok(RecordBatch::new_empty(Arc::clone(&schema_for_stream))) + })) + .boxed(); + + Box::pin(EmptyBatchStream { inner: s, schema }) + } + + #[tokio::test] + async fn yield_less_than_threshold() -> Result<()> { + let count = YIELD_BATCHES - 10; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; + assert_eq!(out.len(), count); + Ok(()) + } + + #[tokio::test] + async fn yield_equal_to_threshold() -> Result<()> { + let count = YIELD_BATCHES; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; + assert_eq!(out.len(), count); + Ok(()) + } + + #[tokio::test] + async fn yield_more_than_threshold() -> Result<()> { + let count = YIELD_BATCHES + 20; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; + assert_eq!(out.len(), count); + Ok(()) + } } diff --git a/parquet-testing b/parquet-testing index 107b36603e05..6e851ddd768d 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit 107b36603e051aee26bd93e04b871034f6c756c0 +Subproject commit 6e851ddd768d6af741c7b15dc594874399fc3cff From 3ff9252761743e511027b868840af58e526c680c Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 30 May 2025 06:25:56 -0400 Subject: [PATCH 10/74] Use existing RecordBatchStreamAdapter --- datafusion/physical-plan/src/stream.rs | 25 +------------------------ parquet-testing | 2 +- 2 files changed, 2 insertions(+), 25 deletions(-) diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index 57a829d3e69d..160628cf0d38 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -716,29 +716,6 @@ mod test { } } - /// A tiny adapter that turns any `Stream>` - /// into a `RecordBatchStream` by carrying along a schema. - struct EmptyBatchStream { - inner: Pin> + Send>>, - schema: SchemaRef, - } - - impl Stream for EmptyBatchStream { - type Item = Result; - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - Pin::new(&mut self.inner).poll_next(cx) - } - } - - impl RecordBatchStream for EmptyBatchStream { - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } - } - /// Helper: construct a SendableRecordBatchStream containing `n` empty batches fn make_empty_batches(n: usize) -> SendableRecordBatchStream { let schema: SchemaRef = Arc::new(Schema::empty()); @@ -750,7 +727,7 @@ mod test { })) .boxed(); - Box::pin(EmptyBatchStream { inner: s, schema }) + Box::pin(RecordBatchStreamAdapter::new(schema, s)) } #[tokio::test] diff --git a/parquet-testing b/parquet-testing index 107b36603e05..6e851ddd768d 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit 107b36603e051aee26bd93e04b871034f6c756c0 +Subproject commit 6e851ddd768d6af741c7b15dc594874399fc3cff From 5547c3c9ec54414ab21742c1d45f6182cf72ab5f Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 31 May 2025 16:22:06 +0800 Subject: [PATCH 11/74] Add timeout testing for cancellation --- .../tests/execution/infinite_agg_cancel.rs | 167 ++++++++++++++++++ datafusion/core/tests/execution/mod.rs | 1 + parquet-testing | 2 +- 3 files changed, 169 insertions(+), 1 deletion(-) create mode 100644 datafusion/core/tests/execution/infinite_agg_cancel.rs diff --git a/datafusion/core/tests/execution/infinite_agg_cancel.rs b/datafusion/core/tests/execution/infinite_agg_cancel.rs new file mode 100644 index 000000000000..5e628d3a2619 --- /dev/null +++ b/datafusion/core/tests/execution/infinite_agg_cancel.rs @@ -0,0 +1,167 @@ +use arrow::array::{Int64Array, RecordBatch}; +use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; +use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; +use datafusion::functions_aggregate::sum; +use datafusion::physical_expr::aggregate::AggregateExprBuilder; +use datafusion::physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion::physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; +use datafusion::prelude::SessionContext; +use datafusion::{common, physical_plan}; +use futures::{Stream, StreamExt}; +use std::any::Any; +use std::error::Error; +use std::fmt::Formatter; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +struct InfiniteStream { + batch: RecordBatch, + poll_count: usize, +} + +impl RecordBatchStream for InfiniteStream { + fn schema(&self) -> SchemaRef { + self.batch.schema() + } +} + +impl Stream for InfiniteStream { + type Item = common::Result; + + fn poll_next( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + self.poll_count += 1; + Poll::Ready(Some(Ok(self.batch.clone()))) + } +} + +#[derive(Debug)] +struct InfiniteExec { + batch: RecordBatch, + properties: PlanProperties, +} + +impl InfiniteExec { + fn new(batch: &RecordBatch) -> Self { + InfiniteExec { + batch: batch.clone(), + properties: PlanProperties::new( + EquivalenceProperties::new(batch.schema().clone()), + Partitioning::UnknownPartitioning(1), + EmissionType::Incremental, + Boundedness::Unbounded { + requires_infinite_memory: false, + }, + ), + } + } +} + +impl DisplayAs for InfiniteExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!(f, "infinite") + } +} + +impl ExecutionPlan for InfiniteExec { + fn name(&self) -> &str { + "infinite" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.batch.schema() + } + + fn properties(&self) -> &PlanProperties { + &self.properties + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + _children: Vec>, + ) -> common::Result> { + Ok(self.clone()) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> common::Result { + Ok(Box::pin(InfiniteStream { + batch: self.batch.clone(), + poll_count: 0, + })) + } +} + +#[tokio::test] +async fn test_infinite_agg_cancel() -> Result<(), Box> { + // 1) build session & schema & sample batch + let session_ctx = SessionContext::new(); + let schema = Arc::new(Schema::new(Fields::from(vec![Field::new( + "value", + DataType::Int64, + false, + )]))); + let mut builder = Int64Array::builder(8192); + for v in 0..8192 { + builder.append_value(v); + } + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(builder.finish())])?; + + // 2) set up the infinite source + aggregation + let inf = Arc::new(InfiniteExec::new(&batch)); + let aggr = Arc::new(AggregateExec::try_new( + AggregateMode::Single, + PhysicalGroupBy::new(vec![], vec![], vec![]), + vec![Arc::new( + AggregateExprBuilder::new( + sum::sum_udaf(), + vec![Arc::new( + datafusion::physical_expr::expressions::Column::new_with_schema( + "value", &schema, + )?, + )], + ) + .schema(inf.schema()) + .alias("sum") + .build()?, + )], + vec![None], + inf.clone(), + inf.schema(), + )?); + + // 3) get the stream + let mut stream = physical_plan::execute_stream(aggr, session_ctx.task_ctx())?; + const TIMEOUT: u64 = 1; + + // 4) drive the stream inline in select! + let result = tokio::select! { + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { + None + } +}; + + assert!(result.is_none(), "Expected timeout, but got a result"); + Ok(()) +} diff --git a/datafusion/core/tests/execution/mod.rs b/datafusion/core/tests/execution/mod.rs index 8169db1a4611..1ddca928cd97 100644 --- a/datafusion/core/tests/execution/mod.rs +++ b/datafusion/core/tests/execution/mod.rs @@ -16,3 +16,4 @@ // under the License. mod logical_plan; +mod infinite_agg_cancel; diff --git a/parquet-testing b/parquet-testing index 6e851ddd768d..107b36603e05 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit 6e851ddd768d6af741c7b15dc594874399fc3cff +Subproject commit 107b36603e051aee26bd93e04b871034f6c756c0 From aeac0ef6981faa6093270601831f7a99b594d612 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 31 May 2025 16:30:12 +0800 Subject: [PATCH 12/74] fmt --- datafusion/core/tests/execution/infinite_agg_cancel.rs | 10 +++++----- datafusion/core/tests/execution/mod.rs | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_agg_cancel.rs b/datafusion/core/tests/execution/infinite_agg_cancel.rs index 5e628d3a2619..de42da00da1e 100644 --- a/datafusion/core/tests/execution/infinite_agg_cancel.rs +++ b/datafusion/core/tests/execution/infinite_agg_cancel.rs @@ -156,11 +156,11 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { // 4) drive the stream inline in select! let result = tokio::select! { - batch_opt = stream.next() => batch_opt, - _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { - None - } -}; + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { + None + } + }; assert!(result.is_none(), "Expected timeout, but got a result"); Ok(()) diff --git a/datafusion/core/tests/execution/mod.rs b/datafusion/core/tests/execution/mod.rs index 1ddca928cd97..92b79b2e4f44 100644 --- a/datafusion/core/tests/execution/mod.rs +++ b/datafusion/core/tests/execution/mod.rs @@ -15,5 +15,5 @@ // specific language governing permissions and limitations // under the License. -mod logical_plan; mod infinite_agg_cancel; +mod logical_plan; From 6d56b78c3696dcee1b01e7d172baf0eb4fd9af6d Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 31 May 2025 16:34:03 +0800 Subject: [PATCH 13/74] add license --- .../core/tests/execution/infinite_agg_cancel.rs | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/datafusion/core/tests/execution/infinite_agg_cancel.rs b/datafusion/core/tests/execution/infinite_agg_cancel.rs index de42da00da1e..af57a3158bdc 100644 --- a/datafusion/core/tests/execution/infinite_agg_cancel.rs +++ b/datafusion/core/tests/execution/infinite_agg_cancel.rs @@ -1,3 +1,20 @@ +// 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::{Int64Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; From 4ddd1e53ff04c81a9fd0573827ef1ac51a5bb907 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 31 May 2025 18:45:35 +0800 Subject: [PATCH 14/74] Support sort exec for cancellation --- ...inite_agg_cancel.rs => infinite_cancel.rs} | 64 +++++++++++++++++-- datafusion/core/tests/execution/mod.rs | 2 +- datafusion/physical-plan/src/sorts/sort.rs | 7 +- 3 files changed, 64 insertions(+), 9 deletions(-) rename datafusion/core/tests/execution/{infinite_agg_cancel.rs => infinite_cancel.rs} (72%) diff --git a/datafusion/core/tests/execution/infinite_agg_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs similarity index 72% rename from datafusion/core/tests/execution/infinite_agg_cancel.rs rename to datafusion/core/tests/execution/infinite_cancel.rs index af57a3158bdc..2aa323418631 100644 --- a/datafusion/core/tests/execution/infinite_agg_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -17,6 +17,7 @@ use arrow::array::{Int64Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; +use arrow_schema::SortOptions; use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; use datafusion::functions_aggregate::sum; use datafusion::physical_expr::aggregate::AggregateExprBuilder; @@ -30,6 +31,9 @@ use datafusion::physical_plan::{ }; use datafusion::prelude::SessionContext; use datafusion::{common, physical_plan}; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_plan::sorts::sort::SortExec; use futures::{Stream, StreamExt}; use std::any::Any; use std::error::Error; @@ -37,6 +41,7 @@ use std::fmt::Formatter; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; +use tokio::select; struct InfiniteStream { batch: RecordBatch, @@ -152,11 +157,7 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { vec![Arc::new( AggregateExprBuilder::new( sum::sum_udaf(), - vec![Arc::new( - datafusion::physical_expr::expressions::Column::new_with_schema( - "value", &schema, - )?, - )], + vec![Arc::new(Column::new_with_schema("value", &schema)?)], ) .schema(inf.schema()) .alias("sum") @@ -172,7 +173,7 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { const TIMEOUT: u64 = 1; // 4) drive the stream inline in select! - let result = tokio::select! { + let result = select! { batch_opt = stream.next() => batch_opt, _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { None @@ -182,3 +183,54 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { assert!(result.is_none(), "Expected timeout, but got a result"); Ok(()) } + +#[tokio::test] +async fn test_infinite_sort_cancel() -> Result<(), Box> { + // 1) build session & schema & sample batch + let session_ctx = SessionContext::new(); + let schema: SchemaRef = Arc::new(Schema::new(vec![Field::new( + "value", + DataType::Int64, + false, + )])); + let mut builder = Int64Array::builder(8192); + for v in 0..8192 { + builder.append_value(v); + } + let array = builder.finish(); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(array)])?; + + // 2) set up the infinite source + let inf = Arc::new(InfiniteExec::new(&batch)); + + // 3) set up a SortExec that will never finish because input is infinite + let sort_options = SortOptions { + descending: false, + nulls_first: true, + }; + let sort_expr = PhysicalSortExpr::new( + Arc::new(Column::new_with_schema("value", &schema)?), + sort_options, + ); + // LexOrdering is just Vec + let lex_ordering: datafusion::physical_expr::LexOrdering = vec![sort_expr].into(); + let sort_exec = Arc::new(SortExec::new(lex_ordering, inf.clone())); + + // 4) get the stream + let mut stream = physical_plan::execute_stream(sort_exec, session_ctx.task_ctx())?; + const TIMEOUT: u64 = 1; + + // 5) drive the stream inline in select! + let result = select! { + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { + None + } + }; + + assert!( + result.is_none(), + "Expected timeout for sort, but got a result" + ); + Ok(()) +} diff --git a/datafusion/core/tests/execution/mod.rs b/datafusion/core/tests/execution/mod.rs index 92b79b2e4f44..333a695dca8e 100644 --- a/datafusion/core/tests/execution/mod.rs +++ b/datafusion/core/tests/execution/mod.rs @@ -15,5 +15,5 @@ // specific language governing permissions and limitations // under the License. -mod infinite_agg_cancel; +mod infinite_cancel; mod logical_plan; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 683983d9e697..1b55ca4ef616 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -36,7 +36,7 @@ use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::get_record_batch_memory_size; use crate::spill::in_progress_spill_file::InProgressSpillFile; use crate::spill::spill_manager::SpillManager; -use crate::stream::RecordBatchStreamAdapter; +use crate::stream::{RecordBatchStreamAdapter, YieldStream}; use crate::topk::TopK; use crate::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, @@ -1093,7 +1093,10 @@ impl ExecutionPlan for SortExec { ) -> Result { trace!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); - let mut input = self.input.execute(partition, Arc::clone(&context))?; + let input = self.input.execute(partition, Arc::clone(&context))?; + + // Yield control back to tokio after a certain number of batches so it can check for cancellation. + let mut input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; let execution_options = &context.session_config().options().execution; From b2ffec78e8b2634d834d9119c4eb8c0917fa1282 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 2 Jun 2025 11:40:40 +0800 Subject: [PATCH 15/74] poc: unified yield exec for leaf node --- .../core/tests/execution/infinite_cancel.rs | 25 ++- datafusion/physical-optimizer/src/lib.rs | 1 + .../physical-optimizer/src/optimizer.rs | 2 + .../src/wrap_leaves_cancellation.rs | 72 +++++++ .../src/aggregates/no_grouping.rs | 4 +- .../physical-plan/src/aggregates/row_hash.rs | 5 +- datafusion/physical-plan/src/lib.rs | 1 + datafusion/physical-plan/src/sorts/sort.rs | 7 +- datafusion/physical-plan/src/stream.rs | 106 ---------- datafusion/physical-plan/src/yield_stream.rs | 185 ++++++++++++++++++ 10 files changed, 289 insertions(+), 119 deletions(-) create mode 100644 datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs create mode 100644 datafusion/physical-plan/src/yield_stream.rs diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index 2aa323418631..4797057f2309 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -42,6 +42,9 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; use tokio::select; +use datafusion_common::config::ConfigOptions; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_optimizer::wrap_leaves_cancellation::WrapLeaves; struct InfiniteStream { batch: RecordBatch, @@ -168,11 +171,15 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { inf.schema(), )?); - // 3) get the stream - let mut stream = physical_plan::execute_stream(aggr, session_ctx.task_ctx())?; + // 3) optimize the plan with WrapLeaves to auto-insert Yield + let optimized = WrapLeaves::new() + .optimize(aggr.clone(), &ConfigOptions::new())?; + + // 4) get the stream + let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; const TIMEOUT: u64 = 1; - // 4) drive the stream inline in select! + // 5) drive the stream inline in select! let result = select! { batch_opt = stream.next() => batch_opt, _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { @@ -216,11 +223,17 @@ async fn test_infinite_sort_cancel() -> Result<(), Box> { let lex_ordering: datafusion::physical_expr::LexOrdering = vec![sort_expr].into(); let sort_exec = Arc::new(SortExec::new(lex_ordering, inf.clone())); - // 4) get the stream - let mut stream = physical_plan::execute_stream(sort_exec, session_ctx.task_ctx())?; + + // 4) optimize the plan with WrapLeaves to auto-insert Yield + let optimized = WrapLeaves::new() + .optimize(sort_exec.clone(), &ConfigOptions::new())?; + + // 5) get the stream + let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; const TIMEOUT: u64 = 1; - // 5) drive the stream inline in select! + + // 6) drive the stream inline in select! let result = select! { batch_opt = stream.next() => batch_opt, _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 5a43d7118d63..f9d868ad207a 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -41,5 +41,6 @@ pub mod sanity_checker; pub mod topk_aggregation; pub mod update_aggr_exprs; pub mod utils; +pub mod wrap_leaves_cancellation; pub use optimizer::PhysicalOptimizerRule; diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index 432ac35ebc23..e483abd92474 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -35,6 +35,7 @@ use crate::sanity_checker::SanityCheckPlan; use crate::topk_aggregation::TopKAggregation; use crate::update_aggr_exprs::OptimizeAggregateOrder; +use crate::wrap_leaves_cancellation::WrapLeaves; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_physical_plan::ExecutionPlan; @@ -137,6 +138,7 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), + Arc::new(WrapLeaves::new()), // The SanityCheckPlan rule checks whether the order and // distribution requirements of each node in the plan // is satisfied. It will also reject non-runnable query diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs new file mode 100644 index 000000000000..537a56ffcf2c --- /dev/null +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -0,0 +1,72 @@ +use crate::PhysicalOptimizerRule; +use datafusion_common::config::ConfigOptions; +use datafusion_common::Result; +use datafusion_physical_plan::yield_stream::YieldStreamExec; +use datafusion_physical_plan::ExecutionPlan; +use std::fmt::{Debug, Formatter}; +use std::sync::Arc; + +/// `WrapLeaves` is a `PhysicalOptimizerRule` that traverses a physical plan +/// and wraps every leaf node (i.e., an `ExecutionPlan` with no children) +/// inside a `YieldStreamExec`. This ensures that long-running leaf operators +/// periodically yield back to the executor and participate in cancellation checks. +pub struct WrapLeaves {} + +impl WrapLeaves { + /// Create a new instance of the WrapLeaves rule. + pub fn new() -> Self { + Self {} + } + + /// Recursively walk the plan: + /// - If `plan.children_any().is_empty()`, it’s a leaf, so wrap it. + /// - Otherwise, recurse into its children, rebuild the node with + /// `with_new_children_any(...)`, and return that. + fn wrap_recursive( + &self, + plan: Arc, + ) -> Result> { + let children = plan.children(); + if children.is_empty() { + // Leaf node: wrap it in `YieldStreamExec` + let wrapped = Arc::new(YieldStreamExec::new(plan.clone())); + Ok(wrapped) + } else { + // Non-leaf: first process all children recursively + let mut new_children = Vec::with_capacity(children.len()); + for child in children { + let wrapped_child = self.wrap_recursive(child.clone())?; + new_children.push(wrapped_child); + } + // Rebuild this node with the new children + let new_plan = plan.with_new_children(new_children)?; + Ok(new_plan) + } + } +} + +impl Debug for WrapLeaves { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + todo!() + } +} + +impl PhysicalOptimizerRule for WrapLeaves { + fn name(&self) -> &str { + "wrap_leaves" + } + + /// Apply the rule by calling `wrap_recursive` on the root plan. + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + self.wrap_recursive(plan) + } + + /// Wrapping leaves does not change the schema, so this remains true. + fn schema_check(&self) -> bool { + true + } +} diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 0651856a1463..ae40f8969ca9 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -36,7 +36,7 @@ use std::task::{Context, Poll}; use super::AggregateExec; use crate::filter::batch_filter; -use crate::stream::YieldStream; +use crate::yield_stream::YieldStream; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use futures::stream::{Stream, StreamExt}; @@ -79,7 +79,7 @@ impl AggregateStream { let input = agg.input.execute(partition, Arc::clone(&context))?; // Yield control back to tokio after a certain number of batches so it can check for cancellation. - let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; + //let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; let filter_expressions = match agg.mode { diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index c10aecb4cc3a..2bc32af44370 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -31,7 +31,7 @@ use crate::metrics::{BaselineMetrics, MetricBuilder, RecordOutput}; use crate::sorts::sort::sort_batch; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::spill_manager::SpillManager; -use crate::stream::{RecordBatchStreamAdapter, YieldStream}; +use crate::stream::RecordBatchStreamAdapter; use crate::{aggregates, metrics, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; @@ -49,6 +49,7 @@ use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; use super::order::GroupOrdering; use super::AggregateExec; +use crate::yield_stream::YieldStream; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::ready; @@ -450,7 +451,7 @@ impl GroupedHashAggregateStream { let input = agg.input.execute(partition, Arc::clone(&context))?; // Yield control back to tokio after a certain number of batches so it can check for cancellation. - let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; + //let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index ba423f958c78..5d63ccdc13a8 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -94,3 +94,4 @@ pub mod udaf { pub mod coalesce; #[cfg(any(test, feature = "bench"))] pub mod test; +pub mod yield_stream; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 1b55ca4ef616..97029c54e305 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -36,7 +36,7 @@ use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::get_record_batch_memory_size; use crate::spill::in_progress_spill_file::InProgressSpillFile; use crate::spill::spill_manager::SpillManager; -use crate::stream::{RecordBatchStreamAdapter, YieldStream}; +use crate::stream::RecordBatchStreamAdapter; use crate::topk::TopK; use crate::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, @@ -57,6 +57,7 @@ use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::{StreamExt, TryStreamExt}; use log::{debug, trace}; +use crate::yield_stream::YieldStream; struct ExternalSorterMetrics { /// metrics @@ -1093,10 +1094,10 @@ impl ExecutionPlan for SortExec { ) -> Result { trace!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); - let input = self.input.execute(partition, Arc::clone(&context))?; + let mut input = self.input.execute(partition, Arc::clone(&context))?; // Yield control back to tokio after a certain number of batches so it can check for cancellation. - let mut input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; + //let mut input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; let execution_options = &context.session_config().options().execution; diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index 160628cf0d38..08013efcd2ae 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -522,71 +522,6 @@ impl Stream for ObservedStream { } } -/// Number of batches to yield before voluntarily returning Pending. -/// This allows long-running operators to periodically yield control -/// back to the executor (e.g., to handle cancellation). -const YIELD_BATCHES: usize = 64; - -/// A stream that yields batches of data, yielding control back to the executor every `YIELD_BATCHES` batches -/// -/// This can be useful to allow operators that might not yield to check for cancellation -pub struct YieldStream { - inner: SendableRecordBatchStream, - batches_processed: usize, - buffer: Option>, -} - -impl YieldStream { - pub fn new(inner: SendableRecordBatchStream) -> Self { - Self { - inner, - batches_processed: 0, - buffer: None, - } - } -} - -// Stream> to poll_next_unpin -impl Stream for YieldStream { - type Item = Result; - - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - let this = &mut *self; - - if let Some(batch) = this.buffer.take() { - return Poll::Ready(Some(batch)); - } - - match this.inner.poll_next_unpin(cx) { - Poll::Ready(Some(Ok(batch))) => { - this.batches_processed += 1; - if this.batches_processed >= YIELD_BATCHES { - this.batches_processed = 0; - // We need to buffer the batch when we return Poll::Pending, - // so that we can return it on the next poll. - // Otherwise, the next poll will miss the batch and return None. - this.buffer = Some(Ok(batch)); - cx.waker().wake_by_ref(); - Poll::Pending - } else { - Poll::Ready(Some(Ok(batch))) - } - } - other => other, - } - } -} - -// RecordBatchStream schema() -impl RecordBatchStream for YieldStream { - fn schema(&self) -> Arc { - self.inner.schema() - } -} - #[cfg(test)] mod test { use super::*; @@ -715,45 +650,4 @@ mod test { ); } } - - /// Helper: construct a SendableRecordBatchStream containing `n` empty batches - fn make_empty_batches(n: usize) -> SendableRecordBatchStream { - let schema: SchemaRef = Arc::new(Schema::empty()); - let schema_for_stream = Arc::clone(&schema); - - let s = - stream::iter((0..n).map(move |_| { - Ok(RecordBatch::new_empty(Arc::clone(&schema_for_stream))) - })) - .boxed(); - - Box::pin(RecordBatchStreamAdapter::new(schema, s)) - } - - #[tokio::test] - async fn yield_less_than_threshold() -> Result<()> { - let count = YIELD_BATCHES - 10; - let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; - assert_eq!(out.len(), count); - Ok(()) - } - - #[tokio::test] - async fn yield_equal_to_threshold() -> Result<()> { - let count = YIELD_BATCHES; - let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; - assert_eq!(out.len(), count); - Ok(()) - } - - #[tokio::test] - async fn yield_more_than_threshold() -> Result<()> { - let count = YIELD_BATCHES + 20; - let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; - assert_eq!(out.len(), count); - Ok(()) - } } diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs new file mode 100644 index 000000000000..84652182ffe9 --- /dev/null +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -0,0 +1,185 @@ +use std::any::Any; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use crate::{ + DisplayAs, DisplayFormatType, ExecutionPlan, + PlanProperties, RecordBatchStream, SendableRecordBatchStream, +}; +use arrow::record_batch::RecordBatch; +use arrow_schema::{Schema, SchemaRef}; +use datafusion_common::Result; +use datafusion_execution::TaskContext; +use futures::{stream, Stream, StreamExt, TryStreamExt}; +use crate::stream::RecordBatchStreamAdapter; + +/// Number of batches to yield before voluntarily returning Pending. +/// This allows long-running operators to periodically yield control +/// back to the executor (e.g., to handle cancellation). +const YIELD_BATCHES: usize = 64; + +/// A stream that yields batches of data, yielding control back to the executor every `YIELD_BATCHES` batches +/// +/// This can be useful to allow operators that might not yield to check for cancellation +pub struct YieldStream { + inner: SendableRecordBatchStream, + batches_processed: usize, + buffer: Option>, +} + +impl YieldStream { + pub fn new(inner: SendableRecordBatchStream) -> Self { + Self { + inner, + batches_processed: 0, + buffer: None, + } + } +} + +// Stream> to poll_next_unpin +impl Stream for YieldStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let this = &mut *self; + + if let Some(batch) = this.buffer.take() { + return Poll::Ready(Some(batch)); + } + + match this.inner.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + this.batches_processed += 1; + if this.batches_processed >= YIELD_BATCHES { + this.batches_processed = 0; + // We need to buffer the batch when we return Poll::Pending, + // so that we can return it on the next poll. + // Otherwise, the next poll will miss the batch and return None. + this.buffer = Some(Ok(batch)); + cx.waker().wake_by_ref(); + Poll::Pending + } else { + Poll::Ready(Some(Ok(batch))) + } + } + other => other, + } + } +} + +// RecordBatchStream schema() +impl RecordBatchStream for YieldStream { + fn schema(&self) -> Arc { + self.inner.schema() + } +} + +#[derive(Debug)] +pub struct YieldStreamExec { + child: Arc, + + properties: PlanProperties, +} + +impl YieldStreamExec { + pub fn new(child: Arc) -> Self { + let properties = child.properties().clone(); + Self { child, properties } + } +} + +impl DisplayAs for YieldStreamExec { + fn fmt_as( + &self, + _t: DisplayFormatType, + f: &mut std::fmt::Formatter<'_>, + ) -> std::fmt::Result { + write!(f, "yield({})", self.child.name()) + } +} + +impl ExecutionPlan for YieldStreamExec { + fn name(&self) -> &str { + "yield_stream_exec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> Arc { + self.child.schema() + } + + fn properties(&self) -> &PlanProperties { + &self.properties + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + Ok(Arc::new(YieldStreamExec::new(children[0].clone()))) + } + + fn execute( + &self, + partition: usize, + task_ctx: Arc, + ) -> Result { + let child_stream = self.child.execute(partition, task_ctx.clone())?; + let yield_stream = YieldStream::new(child_stream); + Ok(Box::pin(yield_stream)) + } +} + + +/// Helper: construct a SendableRecordBatchStream containing `n` empty batches +fn make_empty_batches(n: usize) -> SendableRecordBatchStream { + let schema: SchemaRef = Arc::new(Schema::empty()); + let schema_for_stream = Arc::clone(&schema); + + let s = + stream::iter((0..n).map(move |_| { + Ok(RecordBatch::new_empty(Arc::clone(&schema_for_stream))) + })) + .boxed(); + + Box::pin(RecordBatchStreamAdapter::new(schema, s)) +} + +#[tokio::test] +async fn yield_less_than_threshold() -> Result<()> { + let count = YIELD_BATCHES - 10; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; + assert_eq!(out.len(), count); + Ok(()) +} + +#[tokio::test] +async fn yield_equal_to_threshold() -> Result<()> { + let count = YIELD_BATCHES; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; + assert_eq!(out.len(), count); + Ok(()) +} + +#[tokio::test] +async fn yield_more_than_threshold() -> Result<()> { + let count = YIELD_BATCHES + 20; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; + assert_eq!(out.len(), count); + Ok(()) +} \ No newline at end of file From 4587c3b82305968c7a2cda256290a70ca32dec91 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 2 Jun 2025 20:34:18 +0800 Subject: [PATCH 16/74] polish code phase 1 --- .../core/src/execution/session_state.rs | 2 +- .../core/tests/execution/infinite_cancel.rs | 15 +-- .../src/wrap_leaves_cancellation.rs | 54 ++++++--- .../src/aggregates/no_grouping.rs | 4 - .../physical-plan/src/aggregates/row_hash.rs | 4 - datafusion/physical-plan/src/sorts/sort.rs | 4 - datafusion/physical-plan/src/stream.rs | 1 - datafusion/physical-plan/src/yield_stream.rs | 112 +++++++++++------- 8 files changed, 114 insertions(+), 82 deletions(-) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 8aa812cc5258..0e6408b531fe 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -2177,7 +2177,7 @@ mod tests { let displayable = DisplayableExecutionPlan::new(got.as_ref()); assert_eq!( displayable.indent(false).to_string(), - "ProjectionExec: expr=[0 as count(*)]\n PlaceholderRowExec\n" + "ProjectionExec: expr=[0 as count(*)]\n YieldStreamExec\n PlaceholderRowExec\n" ); Ok(()) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index 4797057f2309..57ff42aca772 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -31,8 +31,11 @@ use datafusion::physical_plan::{ }; use datafusion::prelude::SessionContext; use datafusion::{common, physical_plan}; +use datafusion_common::config::ConfigOptions; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_optimizer::wrap_leaves_cancellation::WrapLeaves; +use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::sorts::sort::SortExec; use futures::{Stream, StreamExt}; use std::any::Any; @@ -42,9 +45,6 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; use tokio::select; -use datafusion_common::config::ConfigOptions; -use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_optimizer::wrap_leaves_cancellation::WrapLeaves; struct InfiniteStream { batch: RecordBatch, @@ -172,8 +172,7 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { )?); // 3) optimize the plan with WrapLeaves to auto-insert Yield - let optimized = WrapLeaves::new() - .optimize(aggr.clone(), &ConfigOptions::new())?; + let optimized = WrapLeaves::new().optimize(aggr.clone(), &ConfigOptions::new())?; // 4) get the stream let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; @@ -223,16 +222,14 @@ async fn test_infinite_sort_cancel() -> Result<(), Box> { let lex_ordering: datafusion::physical_expr::LexOrdering = vec![sort_expr].into(); let sort_exec = Arc::new(SortExec::new(lex_ordering, inf.clone())); - // 4) optimize the plan with WrapLeaves to auto-insert Yield - let optimized = WrapLeaves::new() - .optimize(sort_exec.clone(), &ConfigOptions::new())?; + let optimized = + WrapLeaves::new().optimize(sort_exec.clone(), &ConfigOptions::new())?; // 5) get the stream let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; const TIMEOUT: u64 = 1; - // 6) drive the stream inline in select! let result = select! { batch_opt = stream.next() => batch_opt, diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index 537a56ffcf2c..d7fa3b19c037 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -1,15 +1,17 @@ use crate::PhysicalOptimizerRule; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; +use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::ExecutionPlan; use std::fmt::{Debug, Formatter}; use std::sync::Arc; /// `WrapLeaves` is a `PhysicalOptimizerRule` that traverses a physical plan -/// and wraps every leaf node (i.e., an `ExecutionPlan` with no children) -/// inside a `YieldStreamExec`. This ensures that long-running leaf operators -/// periodically yield back to the executor and participate in cancellation checks. +/// and, for every operator whose `emission_type` is `Final`, wraps its direct +/// children inside a `YieldStreamExec`. This ensures that pipeline‐breaking +/// operators (i.e. those with `Final` emission) have a “yield point” immediately +/// upstream, without having to wait until the leaves. pub struct WrapLeaves {} impl WrapLeaves { @@ -19,35 +21,57 @@ impl WrapLeaves { } /// Recursively walk the plan: - /// - If `plan.children_any().is_empty()`, it’s a leaf, so wrap it. - /// - Otherwise, recurse into its children, rebuild the node with - /// `with_new_children_any(...)`, and return that. + /// - If `plan.children()` is empty, return it unchanged. + /// - If `plan.properties().emission_type == EmissionType::Final`, wrap each + /// direct child in `YieldStreamExec`, then recurse into that wrapper. + /// - Otherwise, recurse into the children normally (without wrapping). + #[allow(clippy::only_used_in_recursion)] fn wrap_recursive( &self, plan: Arc, ) -> Result> { let children = plan.children(); if children.is_empty() { - // Leaf node: wrap it in `YieldStreamExec` - let wrapped = Arc::new(YieldStreamExec::new(plan.clone())); - Ok(wrapped) + // Leaf: no changes + return Ok(plan); + } + + // Recurse into children depending on emission_type + if plan.properties().emission_type == EmissionType::Final { + // For Final‐emission nodes, wrap each direct child in YieldStreamExec + let mut new_children = Vec::with_capacity(children.len()); + for child in children { + // Wrap the immediate child + let wrapped_child = Arc::new(YieldStreamExec::new(Arc::clone(child))); + // Then recurse into the wrapped child, in case there are deeper Final nodes + let rec_wrapped = self.wrap_recursive(wrapped_child)?; + new_children.push(rec_wrapped); + } + // Rebuild this node with its newly wrapped children + let new_plan = plan.with_new_children(new_children)?; + Ok(new_plan) } else { - // Non-leaf: first process all children recursively + // Non‐Final: just recurse into children normally let mut new_children = Vec::with_capacity(children.len()); for child in children { - let wrapped_child = self.wrap_recursive(child.clone())?; - new_children.push(wrapped_child); + let rec_wrapped = self.wrap_recursive(Arc::clone(child))?; + new_children.push(rec_wrapped); } - // Rebuild this node with the new children let new_plan = plan.with_new_children(new_children)?; Ok(new_plan) } } } +impl Default for WrapLeaves { + fn default() -> Self { + Self::new() + } +} + impl Debug for WrapLeaves { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - todo!() + f.debug_struct("WrapLeaves").finish() } } @@ -65,7 +89,7 @@ impl PhysicalOptimizerRule for WrapLeaves { self.wrap_recursive(plan) } - /// Wrapping leaves does not change the schema, so this remains true. + /// Since we only add `YieldStreamExec` wrappers (which preserve schema), schema_check remains true. fn schema_check(&self) -> bool { true } diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index ae40f8969ca9..42c39e3e2554 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -36,7 +36,6 @@ use std::task::{Context, Poll}; use super::AggregateExec; use crate::filter::batch_filter; -use crate::yield_stream::YieldStream; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use futures::stream::{Stream, StreamExt}; @@ -78,9 +77,6 @@ impl AggregateStream { let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let input = agg.input.execute(partition, Arc::clone(&context))?; - // Yield control back to tokio after a certain number of batches so it can check for cancellation. - //let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; - let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; let filter_expressions = match agg.mode { AggregateMode::Partial diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 2bc32af44370..587e0dd538ff 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -49,7 +49,6 @@ use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; use super::order::GroupOrdering; use super::AggregateExec; -use crate::yield_stream::YieldStream; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::ready; @@ -450,9 +449,6 @@ impl GroupedHashAggregateStream { let batch_size = context.session_config().batch_size(); let input = agg.input.execute(partition, Arc::clone(&context))?; - // Yield control back to tokio after a certain number of batches so it can check for cancellation. - //let input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; - let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let timer = baseline_metrics.elapsed_compute().timer(); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 97029c54e305..683983d9e697 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -57,7 +57,6 @@ use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::{StreamExt, TryStreamExt}; use log::{debug, trace}; -use crate::yield_stream::YieldStream; struct ExternalSorterMetrics { /// metrics @@ -1096,9 +1095,6 @@ impl ExecutionPlan for SortExec { let mut input = self.input.execute(partition, Arc::clone(&context))?; - // Yield control back to tokio after a certain number of batches so it can check for cancellation. - //let mut input = Box::pin(YieldStream::new(input)) as SendableRecordBatchStream; - let execution_options = &context.session_config().options().execution; trace!("End SortExec's input.execute for partition: {partition}"); diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index 08013efcd2ae..338ac7d048a3 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -531,7 +531,6 @@ mod test { use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::exec_err; - use futures::{stream, TryStreamExt}; fn schema() -> SchemaRef { Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)])) diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index 84652182ffe9..2f025e3e388d 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -1,18 +1,34 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// for additional information regarding copyright ownership. The +// ASF licenses this file to you under the Apache License, Version +// 2.0 (the "License"); you may not use this file except in +// compliance with the License. You may obtain a copy of the +// License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use std::any::Any; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; use crate::{ - DisplayAs, DisplayFormatType, ExecutionPlan, - PlanProperties, RecordBatchStream, SendableRecordBatchStream, + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, }; use arrow::record_batch::RecordBatch; -use arrow_schema::{Schema, SchemaRef}; +use arrow_schema::Schema; use datafusion_common::Result; use datafusion_execution::TaskContext; -use futures::{stream, Stream, StreamExt, TryStreamExt}; -use crate::stream::RecordBatchStreamAdapter; +use futures::Stream; /// Number of batches to yield before voluntarily returning Pending. /// This allows long-running operators to periodically yield control @@ -52,7 +68,8 @@ impl Stream for YieldStream { return Poll::Ready(Some(batch)); } - match this.inner.poll_next_unpin(cx) { + // Instead of `poll_next_unpin`, use `Pin::new(&mut this.inner).poll_next(cx)` + match Pin::new(&mut this.inner).poll_next(cx) { Poll::Ready(Some(Ok(batch))) => { this.batches_processed += 1; if this.batches_processed >= YIELD_BATCHES { @@ -99,7 +116,7 @@ impl DisplayAs for YieldStreamExec { _t: DisplayFormatType, f: &mut std::fmt::Formatter<'_>, ) -> std::fmt::Result { - write!(f, "yield({})", self.child.name()) + write!(f, "YieldStreamExec") } } @@ -128,7 +145,8 @@ impl ExecutionPlan for YieldStreamExec { self: Arc, children: Vec>, ) -> Result> { - Ok(Arc::new(YieldStreamExec::new(children[0].clone()))) + // Use Arc::clone on children[0] rather than calling clone() directly + Ok(Arc::new(YieldStreamExec::new(Arc::clone(&children[0])))) } fn execute( @@ -136,50 +154,56 @@ impl ExecutionPlan for YieldStreamExec { partition: usize, task_ctx: Arc, ) -> Result { - let child_stream = self.child.execute(partition, task_ctx.clone())?; + let child_stream = self.child.execute(partition, Arc::clone(&task_ctx))?; let yield_stream = YieldStream::new(child_stream); Ok(Box::pin(yield_stream)) } } +#[cfg(test)] +mod tests { + use super::*; + use crate::stream::RecordBatchStreamAdapter; + use arrow_schema::SchemaRef; + use futures::{stream, StreamExt}; -/// Helper: construct a SendableRecordBatchStream containing `n` empty batches -fn make_empty_batches(n: usize) -> SendableRecordBatchStream { - let schema: SchemaRef = Arc::new(Schema::empty()); - let schema_for_stream = Arc::clone(&schema); + /// Helper: construct a SendableRecordBatchStream containing `n` empty batches + fn make_empty_batches(n: usize) -> SendableRecordBatchStream { + let schema: SchemaRef = Arc::new(Schema::empty()); + let schema_for_stream = Arc::clone(&schema); - let s = - stream::iter((0..n).map(move |_| { - Ok(RecordBatch::new_empty(Arc::clone(&schema_for_stream))) - })) - .boxed(); + let s = + stream::iter((0..n).map(move |_| { + Ok(RecordBatch::new_empty(Arc::clone(&schema_for_stream))) + })); - Box::pin(RecordBatchStreamAdapter::new(schema, s)) -} + Box::pin(RecordBatchStreamAdapter::new(schema, s)) + } -#[tokio::test] -async fn yield_less_than_threshold() -> Result<()> { - let count = YIELD_BATCHES - 10; - let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; - assert_eq!(out.len(), count); - Ok(()) -} + #[tokio::test] + async fn yield_less_than_threshold() -> Result<()> { + let count = YIELD_BATCHES - 10; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).collect::>().await; + assert_eq!(out.len(), count); + Ok(()) + } -#[tokio::test] -async fn yield_equal_to_threshold() -> Result<()> { - let count = YIELD_BATCHES; - let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; - assert_eq!(out.len(), count); - Ok(()) -} + #[tokio::test] + async fn yield_equal_to_threshold() -> Result<()> { + let count = YIELD_BATCHES; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).collect::>().await; + assert_eq!(out.len(), count); + Ok(()) + } -#[tokio::test] -async fn yield_more_than_threshold() -> Result<()> { - let count = YIELD_BATCHES + 20; - let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).try_collect::>().await?; - assert_eq!(out.len(), count); - Ok(()) -} \ No newline at end of file + #[tokio::test] + async fn yield_more_than_threshold() -> Result<()> { + let count = YIELD_BATCHES + 20; + let inner = make_empty_batches(count); + let out: Vec<_> = YieldStream::new(inner).collect::>().await; + assert_eq!(out.len(), count); + Ok(()) + } +} From 098b1ecd73510ed93827af684f600e324e36a6e3 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 2 Jun 2025 21:07:56 +0800 Subject: [PATCH 17/74] Add license --- .../src/wrap_leaves_cancellation.rs | 53 ++++++++++--------- 1 file changed, 27 insertions(+), 26 deletions(-) diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index d7fa3b19c037..1f6b6746158a 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -1,7 +1,23 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use crate::PhysicalOptimizerRule; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; -use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::ExecutionPlan; use std::fmt::{Debug, Formatter}; @@ -21,10 +37,9 @@ impl WrapLeaves { } /// Recursively walk the plan: - /// - If `plan.children()` is empty, return it unchanged. - /// - If `plan.properties().emission_type == EmissionType::Final`, wrap each - /// direct child in `YieldStreamExec`, then recurse into that wrapper. - /// - Otherwise, recurse into the children normally (without wrapping). + /// - If `plan.children_any().is_empty()`, it’s a leaf, so wrap it. + /// - Otherwise, recurse into its children, rebuild the node with + /// `with_new_children_any(...)`, and return that. #[allow(clippy::only_used_in_recursion)] fn wrap_recursive( &self, @@ -32,31 +47,17 @@ impl WrapLeaves { ) -> Result> { let children = plan.children(); if children.is_empty() { - // Leaf: no changes - return Ok(plan); - } - - // Recurse into children depending on emission_type - if plan.properties().emission_type == EmissionType::Final { - // For Final‐emission nodes, wrap each direct child in YieldStreamExec - let mut new_children = Vec::with_capacity(children.len()); - for child in children { - // Wrap the immediate child - let wrapped_child = Arc::new(YieldStreamExec::new(Arc::clone(child))); - // Then recurse into the wrapped child, in case there are deeper Final nodes - let rec_wrapped = self.wrap_recursive(wrapped_child)?; - new_children.push(rec_wrapped); - } - // Rebuild this node with its newly wrapped children - let new_plan = plan.with_new_children(new_children)?; - Ok(new_plan) + // Leaf node: wrap it in `YieldStreamExec` + let wrapped = Arc::new(YieldStreamExec::new(plan)); + Ok(wrapped) } else { - // Non‐Final: just recurse into children normally + // Non-leaf: first process all children recursively let mut new_children = Vec::with_capacity(children.len()); for child in children { - let rec_wrapped = self.wrap_recursive(Arc::clone(child))?; - new_children.push(rec_wrapped); + let wrapped_child = self.wrap_recursive(Arc::clone(child))?; + new_children.push(wrapped_child); } + // Rebuild this node with the new children let new_plan = plan.with_new_children(new_children)?; Ok(new_plan) } From bc65c1a80b28aa07763e7f9b8b3489bd17478f43 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 2 Jun 2025 21:24:44 +0800 Subject: [PATCH 18/74] Fix testing --- datafusion/core/src/physical_planner.rs | 4 +- datafusion/physical-plan/src/yield_stream.rs | 12 +- .../test_files/agg_func_substitute.slt | 9 +- .../sqllogictest/test_files/aggregate.slt | 112 ++- .../test_files/aggregates_topk.slt | 21 +- datafusion/sqllogictest/test_files/alias.slt | 12 +- datafusion/sqllogictest/test_files/array.slt | 24 +- .../sqllogictest/test_files/arrow_files.slt | 8 +- datafusion/sqllogictest/test_files/binary.slt | 3 +- datafusion/sqllogictest/test_files/copy.slt | 9 +- .../test_files/count_star_rule.slt | 12 +- .../test_files/create_external_table.slt | 11 +- datafusion/sqllogictest/test_files/cse.slt | 33 +- .../sqllogictest/test_files/csv_files.slt | 3 +- datafusion/sqllogictest/test_files/cte.slt | 113 ++- datafusion/sqllogictest/test_files/ddl.slt | 8 +- .../sqllogictest/test_files/dictionary.slt | 9 +- .../sqllogictest/test_files/distinct_on.slt | 3 +- .../sqllogictest/test_files/explain.slt | 159 +++- .../sqllogictest/test_files/explain_tree.slt | 857 ++++++++++++------ datafusion/sqllogictest/test_files/expr.slt | 3 +- .../test_files/filter_without_sort_exec.slt | 18 +- .../sqllogictest/test_files/group_by.slt | 236 +++-- datafusion/sqllogictest/test_files/insert.slt | 12 +- .../test_files/insert_to_external.slt | 12 +- .../sqllogictest/test_files/join.slt.part | 82 +- .../join_disable_repartition_joins.slt | 20 +- datafusion/sqllogictest/test_files/joins.slt | 575 +++++++----- datafusion/sqllogictest/test_files/json.slt | 7 +- datafusion/sqllogictest/test_files/limit.slt | 52 +- .../test_files/listing_table_statistics.slt | 4 +- datafusion/sqllogictest/test_files/map.slt | 3 +- .../test_files/monotonic_projection_test.slt | 24 +- .../sqllogictest/test_files/operator.slt | 24 +- .../sqllogictest/test_files/options.slt | 9 +- datafusion/sqllogictest/test_files/order.slt | 103 ++- .../sqllogictest/test_files/parquet.slt | 21 +- .../test_files/parquet_filter_pushdown.slt | 26 +- .../test_files/parquet_sorted_statistics.slt | 18 +- .../test_files/parquet_statistics.slt | 21 +- .../sqllogictest/test_files/predicates.slt | 60 +- .../sqllogictest/test_files/projection.slt | 7 +- .../test_files/push_down_filter.slt | 46 +- .../sqllogictest/test_files/references.slt | 3 +- .../test_files/regexp/regexp_like.slt | 6 +- .../sqllogictest/test_files/repartition.slt | 9 +- .../test_files/repartition_scan.slt | 26 +- datafusion/sqllogictest/test_files/scalar.slt | 6 +- datafusion/sqllogictest/test_files/select.slt | 70 +- .../sqllogictest/test_files/simplify_expr.slt | 12 +- .../test_files/sort_merge_join.slt | 8 +- datafusion/sqllogictest/test_files/struct.slt | 3 +- .../sqllogictest/test_files/subquery.slt | 48 +- .../sqllogictest/test_files/subquery_sort.slt | 20 +- .../test_files/table_functions.slt | 8 +- datafusion/sqllogictest/test_files/topk.slt | 30 +- datafusion/sqllogictest/test_files/union.slt | 190 ++-- datafusion/sqllogictest/test_files/unnest.slt | 6 +- datafusion/sqllogictest/test_files/window.slt | 228 +++-- 59 files changed, 2262 insertions(+), 1216 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index fbb4250fc4df..454911afcde0 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -3050,8 +3050,10 @@ mod tests { digraph { 1[shape=box label="ProjectionExec: expr=[id@0 + 2 as employee.id + Int32(2)]", tooltip=""] - 2[shape=box label="EmptyExec", tooltip=""] + 2[shape=box label="YieldStreamExec child=EmptyExec", tooltip=""] 1 -> 2 [arrowhead=none, arrowtail=normal, dir=back] + 3[shape=box label="EmptyExec", tooltip=""] + 2 -> 3 [arrowhead=none, arrowtail=normal, dir=back] } // End DataFusion GraphViz Plan "#; diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index 2f025e3e388d..61cdf5f58aab 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -1,10 +1,10 @@ // Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file -// 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 +// 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 // @@ -116,7 +116,7 @@ impl DisplayAs for YieldStreamExec { _t: DisplayFormatType, f: &mut std::fmt::Formatter<'_>, ) -> std::fmt::Result { - write!(f, "YieldStreamExec") + write!(f, "YieldStreamExec child={}", self.child.name()) } } diff --git a/datafusion/sqllogictest/test_files/agg_func_substitute.slt b/datafusion/sqllogictest/test_files/agg_func_substitute.slt index 9aeaaacb1071..d512a77e543d 100644 --- a/datafusion/sqllogictest/test_files/agg_func_substitute.slt +++ b/datafusion/sqllogictest/test_files/agg_func_substitute.slt @@ -50,7 +50,8 @@ physical_plan 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)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------DataSourceExec: 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]], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: 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]], file_type=csv, has_header=true query TT @@ -70,7 +71,8 @@ physical_plan 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)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------DataSourceExec: 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]], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: 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]], file_type=csv, has_header=true query TT EXPLAIN SELECT a, ARRAY_AGG(c ORDER BY c)[1 + 100] as result @@ -89,7 +91,8 @@ physical_plan 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)--------------DataSourceExec: 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]], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: 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]], file_type=csv, 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 52b1e1c22fdf..5d0258b130e1 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -250,7 +250,8 @@ physical_plan 03)----AggregateExec: mode=Partial, gby=[], aggr=[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] 04)------SortExec: expr=[c2@1 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[true] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true # test array_agg_order with list data type statement ok @@ -393,15 +394,20 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[array_agg(DISTINCT a.foo), sum(DISTINCT Int64(1))], ordering_mode=Sorted 06)----------UnionExec 07)------------ProjectionExec: expr=[1 as id, 2 as foo] -08)--------------PlaceholderRowExec -09)------------ProjectionExec: expr=[1 as id, NULL as foo] -10)--------------PlaceholderRowExec -11)------------ProjectionExec: expr=[1 as id, NULL as foo] -12)--------------PlaceholderRowExec -13)------------ProjectionExec: expr=[1 as id, 3 as foo] -14)--------------PlaceholderRowExec -15)------------ProjectionExec: expr=[1 as id, 2 as foo] -16)--------------PlaceholderRowExec +08)--------------YieldStreamExec child=PlaceholderRowExec +09)----------------PlaceholderRowExec +10)------------ProjectionExec: expr=[1 as id, NULL as foo] +11)--------------YieldStreamExec child=PlaceholderRowExec +12)----------------PlaceholderRowExec +13)------------ProjectionExec: expr=[1 as id, NULL as foo] +14)--------------YieldStreamExec child=PlaceholderRowExec +15)----------------PlaceholderRowExec +16)------------ProjectionExec: expr=[1 as id, 3 as foo] +17)--------------YieldStreamExec child=PlaceholderRowExec +18)----------------PlaceholderRowExec +19)------------ProjectionExec: expr=[1 as id, 2 as foo] +20)--------------YieldStreamExec child=PlaceholderRowExec +21)----------------PlaceholderRowExec # FIX: custom absolute values @@ -1130,7 +1136,8 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=4 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; @@ -5294,7 +5301,8 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true # @@ -5319,7 +5327,8 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], file_type=csv, has_header=true query I SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 order by c3 limit 5; @@ -5343,7 +5352,8 @@ physical_plan 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)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query II SELECT c2, c3 FROM aggregate_test_100 group by c2, c3 order by c2, c3 limit 5 offset 4; @@ -5378,7 +5388,8 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true +13)------------------------YieldStreamExec child=DataSourceExec +14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query I SELECT DISTINCT c3 FROM aggregate_test_100 WHERE c3 between 10 and 20 group by c3 order by c3 limit 4; @@ -5404,7 +5415,8 @@ physical_plan 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)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, 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 @@ -5428,7 +5440,8 @@ physical_plan 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)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query II SELECT DISTINCT c3, c2 FROM aggregate_test_100 group by c3, c2 order by c3, c2 limit 3 offset 10; @@ -5452,7 +5465,8 @@ physical_plan 04)------CoalescePartitionsExec 05)--------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=[] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query II SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; @@ -5479,7 +5493,8 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], file_type=csv, has_header=true statement ok set datafusion.optimizer.enable_distinct_aggregation_soft_limit = true; @@ -6078,7 +6093,8 @@ logical_plan 02)--TableScan: empty projection=[col0] physical_plan 01)ProjectionExec: expr=[NULL as min(empty.col0)] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query TT EXPLAIN SELECT MAX(col0) FROM empty; @@ -6088,7 +6104,8 @@ logical_plan 02)--TableScan: empty projection=[col0] physical_plan 01)ProjectionExec: expr=[NULL as max(empty.col0)] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec statement ok DROP TABLE empty; @@ -6308,7 +6325,8 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[last_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 ASC NULLS LAST]] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: 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]], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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]], file_type=csv, has_header=true # test last to first query TT @@ -6322,7 +6340,8 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[first_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 DESC NULLS FIRST]] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: 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]], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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]], file_type=csv, has_header=true # test building plan with aggreagte sum @@ -6395,7 +6414,8 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query error @@ -6545,15 +6565,20 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[last_value(a.foo) ORDER BY [a.foo ASC NULLS LAST], sum(DISTINCT Int64(1))], ordering_mode=Sorted 06)----------UnionExec 07)------------ProjectionExec: expr=[1 as id, 2 as foo] -08)--------------PlaceholderRowExec -09)------------ProjectionExec: expr=[1 as id, 4 as foo] -10)--------------PlaceholderRowExec -11)------------ProjectionExec: expr=[1 as id, 5 as foo] -12)--------------PlaceholderRowExec -13)------------ProjectionExec: expr=[1 as id, 3 as foo] -14)--------------PlaceholderRowExec -15)------------ProjectionExec: expr=[1 as id, 2 as foo] -16)--------------PlaceholderRowExec +08)--------------YieldStreamExec child=PlaceholderRowExec +09)----------------PlaceholderRowExec +10)------------ProjectionExec: expr=[1 as id, 4 as foo] +11)--------------YieldStreamExec child=PlaceholderRowExec +12)----------------PlaceholderRowExec +13)------------ProjectionExec: expr=[1 as id, 5 as foo] +14)--------------YieldStreamExec child=PlaceholderRowExec +15)----------------PlaceholderRowExec +16)------------ProjectionExec: expr=[1 as id, 3 as foo] +17)--------------YieldStreamExec child=PlaceholderRowExec +18)----------------PlaceholderRowExec +19)------------ProjectionExec: expr=[1 as id, 2 as foo] +20)--------------YieldStreamExec child=PlaceholderRowExec +21)----------------PlaceholderRowExec # SortExec is removed if it is coming after one-row producing AggregateExec's having an empty group by expression query TT @@ -6570,7 +6595,8 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(aggregate_test_100.c5)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], file_type=csv, has_header=true statement count 0 drop table aggregate_test_100; @@ -6593,7 +6619,8 @@ logical_plan 02)--TableScan: t projection=[] physical_plan 01)AggregateExec: mode=Single, gby=[], aggr=[count(NULL)] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement count 0 drop table t; @@ -6667,7 +6694,8 @@ logical_plan 02)--TableScan: t projection=[] physical_plan 01)AggregateExec: mode=Single, gby=[], aggr=[count(Int64(1)), count(Int64(2))] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] query II select count(1), count() from t; @@ -6683,7 +6711,8 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(Int64(1)), 2 as count()] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query II select count(1), count(*) from t; @@ -6699,7 +6728,8 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(Int64(1)), 2 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query II select count(), count(*) from t; @@ -6715,7 +6745,8 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(), 2 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query TT explain select count(1) * count(2) from t; @@ -6727,7 +6758,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 * count(Int64(2))@1 as count(Int64(1)) * count(Int64(2))] 02)--AggregateExec: mode=Single, gby=[], aggr=[count(Int64(1)), count(Int64(2))] -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] statement count 0 drop table t; diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index cc1693843848..96a9da7d1665 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -50,7 +50,8 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query TI select * from (select trace_id, MAX(timestamp) max_ts from traces t group by trace_id) where trace_id != 'b' order by max_ts desc limit 3; @@ -114,7 +115,8 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: 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; @@ -131,7 +133,8 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: 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; @@ -148,7 +151,8 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: 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; @@ -165,7 +169,8 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: 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; @@ -239,7 +244,8 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces_utf8view.timestamp)], lim=[4] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] # Also add LargeUtf8 to test PR https://github.com/apache/datafusion/pull/15152 @@ -267,7 +273,8 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces_largeutf8.timestamp)], lim=[4] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/alias.slt b/datafusion/sqllogictest/test_files/alias.slt index 5339179db4c4..780d12d4ffdf 100644 --- a/datafusion/sqllogictest/test_files/alias.slt +++ b/datafusion/sqllogictest/test_files/alias.slt @@ -34,8 +34,10 @@ logical_plan 04)----TableScan: t2 projection=[age] physical_plan 01)CrossJoinExec -02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[0] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)--YieldStreamExec child=DataSourceExec +05)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain select * from ((select id from t1) cross join (select age from t2)) as f(c1, c2); @@ -49,8 +51,10 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as c1, age@1 as c2] 02)--CrossJoinExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table t1; diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index d89ba600d7a6..322b4a540696 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -6034,7 +6034,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec child=LazyMemoryExec +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6063,7 +6064,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec child=LazyMemoryExec +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6092,7 +6094,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec child=LazyMemoryExec +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # FIXME: due to rewrite below not working, this is _extremely_ slow to evaluate # query I @@ -6123,7 +6126,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: array_has([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c], substr(md5(CAST(value@0 AS Utf8)), 1, 32)) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec child=LazyMemoryExec +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6152,7 +6156,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec child=LazyMemoryExec +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6183,7 +6188,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IS NOT NULL OR NULL 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec child=LazyMemoryExec +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # any operator query ? @@ -7196,7 +7202,8 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[true as array_has_all(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(1),Int64(3)))] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec # array containment operator with scalars #2 (arrow at) query BBBBBBB @@ -7219,7 +7226,8 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[true as array_has_all(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(1),Int64(3)))] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec ### Array casting tests diff --git a/datafusion/sqllogictest/test_files/arrow_files.slt b/datafusion/sqllogictest/test_files/arrow_files.slt index 30f322cf98fc..95ee4afe5528 100644 --- a/datafusion/sqllogictest/test_files/arrow_files.slt +++ b/datafusion/sqllogictest/test_files/arrow_files.slt @@ -32,7 +32,9 @@ query TT EXPLAIN SELECT * FROM arrow_simple ---- logical_plan TableScan: arrow_simple projection=[f0, f1, f2] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[f0, f1, f2], file_type=arrow +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[f0, f1, f2], file_type=arrow # correct content query ITB @@ -117,7 +119,9 @@ query TT EXPLAIN SELECT f0 FROM arrow_partitioned WHERE part = 456 ---- logical_plan TableScan: arrow_partitioned projection=[f0], full_filters=[arrow_partitioned.part = Int32(456)] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_arrow/part=456/data.arrow]]}, projection=[f0], file_type=arrow +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_arrow/part=456/data.arrow]]}, projection=[f0], file_type=arrow # Errors in partition filters should be reported diff --git a/datafusion/sqllogictest/test_files/binary.slt b/datafusion/sqllogictest/test_files/binary.slt index 1077c32e46f3..3631967396d6 100644 --- a/datafusion/sqllogictest/test_files/binary.slt +++ b/datafusion/sqllogictest/test_files/binary.slt @@ -185,7 +185,8 @@ logical_plan 02)--TableScan: t projection=[column1] physical_plan 01)ProjectionExec: expr=[column1@0 as column1, column1@0 = 000102 as t.column1 = Binary("0,1,2")] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t_source diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index 5eeb05e814ac..90957b802a24 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -184,7 +184,8 @@ logical_plan 02)--TableScan: source_table projection=[col1, col2] physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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. @@ -198,7 +199,8 @@ logical_plan 02)--TableScan: source_table projection=[col1, col2] physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] # Copy to directory as partitioned files with keep_partition_by_columns enabled query I @@ -656,7 +658,8 @@ logical_plan 03)----TableScan: t projection=[], fetch=10 physical_plan 01)DataSinkExec: sink=CsvSink(file_groups=[]) -02)--DataSourceExec: partitions=1, partition_sizes=[0], fetch=10 +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0], fetch=10 statement count 0 drop table t; diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index d38d3490fed4..15c5f6d2a2c6 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -37,7 +37,8 @@ logical_plan 04)------EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as count()] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query TT EXPLAIN SELECT t1.a, COUNT() FROM t1 GROUP BY t1.a; @@ -53,7 +54,8 @@ physical_plan 04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 0; @@ -72,7 +74,8 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 1; @@ -90,7 +93,8 @@ physical_plan 01)ProjectionExec: expr=[a@0 as a, count(Int64(1)) PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as count_a] 02)--WindowAggExec: wdw=[count(Int64(1)) PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(Int64(1)) PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, 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 }] 03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT a, COUNT() OVER (PARTITION BY a) AS count_a FROM t1 ORDER BY a; diff --git a/datafusion/sqllogictest/test_files/create_external_table.slt b/datafusion/sqllogictest/test_files/create_external_table.slt index 03cb5edb5fcc..7ce64bed2894 100644 --- a/datafusion/sqllogictest/test_files/create_external_table.slt +++ b/datafusion/sqllogictest/test_files/create_external_table.slt @@ -253,7 +253,9 @@ EXPLAIN SELECT id FROM t ORDER BY id ASC; logical_plan 01)Sort: t.id ASC NULLS LAST 02)--TableScan: t projection=[id] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet ## Test a DESC order and verify that output_ordering is ASC from the previous OBRDER BY query TT @@ -264,7 +266,8 @@ logical_plan 02)--TableScan: t projection=[id] physical_plan 01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet statement ok DROP TABLE t; @@ -278,7 +281,9 @@ query TT EXPLAIN SELECT id FROM t; ---- logical_plan TableScan: t projection=[id] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 DESC], file_type=parquet +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 DESC], file_type=parquet statement ok DROP TABLE t; diff --git a/datafusion/sqllogictest/test_files/cse.slt b/datafusion/sqllogictest/test_files/cse.slt index 1af4f14c937e..02e04872f88b 100644 --- a/datafusion/sqllogictest/test_files/cse.slt +++ b/datafusion/sqllogictest/test_files/cse.slt @@ -32,7 +32,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as c1, __common_expr_1@0 as c2] 02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1] -03)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] # Common volatile expression query TT @@ -46,7 +47,8 @@ logical_plan 02)--TableScan: t1 projection=[a] physical_plan 01)ProjectionExec: expr=[a@0 + random() as c1, a@0 + random() as c2] -02)--DataSourceExec: partitions=1, partition_sizes=[0] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Volatile expression with non-volatile common child query TT @@ -62,7 +64,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 + random() as c1, __common_expr_1@0 + random() as c2] 02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1] -03)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] # Volatile expression with non-volatile common children query TT @@ -78,7 +81,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 + random() + __common_expr_2@1 as c1, __common_expr_1@0 + random() + __common_expr_2@1 as c2] 02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1, a@0 + 2 as __common_expr_2] -03)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] # Common short-circuit expression query TT @@ -100,7 +104,8 @@ physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as c1, __common_expr_1@0 as c2, __common_expr_2@1 as c3, __common_expr_2@1 as c4, __common_expr_3@2 as c5, __common_expr_3@2 as c6] 02)--ProjectionExec: expr=[__common_expr_4@0 AND b@1 = 0 as __common_expr_1, __common_expr_4@0 OR b@1 = 0 as __common_expr_2, CASE WHEN __common_expr_4@0 THEN 0 ELSE 1 END as __common_expr_3] 03)----ProjectionExec: expr=[a@0 = 0 as __common_expr_4, b@1 as b] -04)------DataSourceExec: partitions=1, partition_sizes=[0] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[0] # Common children of short-circuit expression query TT @@ -128,7 +133,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 AND b@4 = 0 as c1, __common_expr_1@0 AND b@4 = 1 as c2, b@4 = 2 AND a@3 = 1 as c3, b@4 = 3 AND a@3 = 1 as c4, __common_expr_2@1 OR b@4 = 4 as c5, __common_expr_2@1 OR b@4 = 5 as c6, b@4 = 6 OR a@3 = 3 as c7, b@4 = 7 OR a@3 = 3 as c8, CASE WHEN __common_expr_3@2 THEN 0 ELSE 1 END as c9, CASE WHEN __common_expr_3@2 THEN 0 ELSE 2 END as c10, CASE WHEN b@4 = 8 THEN a@3 + 1 ELSE 0 END as c11, CASE WHEN b@4 = 9 THEN a@3 + 1 ELSE 0 END as c12, CASE WHEN b@4 = 10 THEN 0 ELSE a@3 + 2 END as c13, CASE WHEN b@4 = 11 THEN 0 ELSE a@3 + 2 END as c14] 02)--ProjectionExec: expr=[a@0 = 0 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 = 4 as __common_expr_3, a@0 as a, b@1 as b] -03)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] # Common children of volatile, short-circuit expression query TT @@ -156,7 +162,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 AND b@4 = random() as c1, __common_expr_1@0 AND b@4 = 1 + random() as c2, b@4 = 2 + random() AND a@3 = 1 as c3, b@4 = 3 + random() AND a@3 = 1 as c4, __common_expr_2@1 OR b@4 = 4 + random() as c5, __common_expr_2@1 OR b@4 = 5 + random() as c6, b@4 = 6 + random() OR a@3 = 3 as c7, b@4 = 7 + random() OR a@3 = 3 as c8, CASE WHEN __common_expr_3@2 THEN random() ELSE 1 END as c9, CASE WHEN __common_expr_3@2 THEN random() ELSE 2 END as c10, CASE WHEN b@4 = 8 + random() THEN a@3 + 1 ELSE 0 END as c11, CASE WHEN b@4 = 9 + random() THEN a@3 + 1 ELSE 0 END as c12, CASE WHEN b@4 = 10 + random() THEN 0 ELSE a@3 + 2 END as c13, CASE WHEN b@4 = 11 + random() THEN 0 ELSE a@3 + 2 END as c14] 02)--ProjectionExec: expr=[a@0 = 0 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 = 4 as __common_expr_3, a@0 as a, b@1 as b] -03)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] # Common volatile children of short-circuit expression query TT @@ -174,7 +181,8 @@ logical_plan 02)--TableScan: t1 projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 = random() AND b@1 = 0 as c1, a@0 = random() AND b@1 = 1 as c2, a@0 = 2 + random() OR b@1 = 4 as c3, a@0 = 2 + random() OR b@1 = 5 as c4, CASE WHEN a@0 = 4 + random() THEN 0 ELSE 1 END as c5, CASE WHEN a@0 = 4 + random() THEN 0 ELSE 2 END as c6] -02)--DataSourceExec: partitions=1, partition_sizes=[0] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Surely only once but also conditionally evaluated expressions query TT @@ -194,7 +202,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[(__common_expr_1@0 OR random() = 0) AND __common_expr_2@1 as c1, __common_expr_2@1 AND random() = 0 OR __common_expr_1@0 as c2, CASE WHEN __common_expr_3@2 = 0 THEN __common_expr_3@2 ELSE 0 END as c3, CASE WHEN __common_expr_4@3 = 0 THEN 0 WHEN CAST(__common_expr_4@3 AS Boolean) THEN 0 ELSE 0 END as c4, CASE WHEN __common_expr_5@4 = 0 THEN 0 WHEN random() = 0 THEN __common_expr_5@4 ELSE 0 END as c5, CASE WHEN __common_expr_6@5 = 0 THEN 0 ELSE __common_expr_6@5 END as c6] 02)--ProjectionExec: expr=[a@0 = 1 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 + 3 as __common_expr_3, a@0 + 4 as __common_expr_4, a@0 + 5 as __common_expr_5, a@0 + 6 as __common_expr_6] -03)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] # Surely only once but also conditionally evaluated subexpressions query TT @@ -212,7 +221,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[(__common_expr_1@0 OR random() = 0) AND (__common_expr_2@1 OR random() = 1) as c1, __common_expr_2@1 AND random() = 0 OR __common_expr_1@0 AND random() = 1 as c2, CASE WHEN __common_expr_3@2 = 0 THEN __common_expr_3@2 + random() ELSE 0 END as c3, CASE WHEN __common_expr_4@3 = 0 THEN 0 ELSE __common_expr_4@3 + random() END as c4] 02)--ProjectionExec: expr=[a@0 = 1 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 + 3 as __common_expr_3, a@0 + 4 as __common_expr_4] -03)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] # Only conditionally evaluated expressions query TT @@ -230,4 +240,5 @@ logical_plan 02)--TableScan: t1 projection=[a] physical_plan 01)ProjectionExec: expr=[(random() = 0 OR a@0 = 1) AND a@0 = 2 as c1, random() = 0 AND a@0 = 2 OR a@0 = 1 as c2, CASE WHEN random() = 0 THEN a@0 + 3 ELSE a@0 + 3 END as c3, CASE WHEN random() = 0 THEN 0 WHEN a@0 + 4 = 0 THEN a@0 + 4 ELSE 0 END as c4, CASE WHEN random() = 0 THEN 0 WHEN a@0 + 5 = 0 THEN 0 ELSE a@0 + 5 END as c5, CASE WHEN random() = 0 THEN 0 WHEN random() = 0 THEN a@0 + 6 ELSE a@0 + 6 END as c6] -02)--DataSourceExec: partitions=1, partition_sizes=[0] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] diff --git a/datafusion/sqllogictest/test_files/csv_files.slt b/datafusion/sqllogictest/test_files/csv_files.slt index 5a7fa309dbfa..94700c8e7811 100644 --- a/datafusion/sqllogictest/test_files/csv_files.slt +++ b/datafusion/sqllogictest/test_files/csv_files.slt @@ -165,7 +165,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [int_col@0 ASC NULLS LAST] 02)--SortExec: expr=[int_col@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: 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], file_type=csv, has_header=false +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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], file_type=csv, has_header=false # ensure that correct quote character is used when writing to csv diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 32320a06f4fb..a686e5c1cab5 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -36,7 +36,8 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec # cte_use_same_name_multiple_times statement error DataFusion error: Error during planning: WITH query name "a" specified more than once @@ -114,13 +115,15 @@ logical_plan physical_plan 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 +03)----YieldStreamExec child=PlaceholderRowExec +04)------PlaceholderRowExec +05)--CoalescePartitionsExec +06)----ProjectionExec: expr=[id@0 + 1 as id] +07)------CoalesceBatchesExec: target_batch_size=8192 +08)--------FilterExec: id@0 < 10 +09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------YieldStreamExec child=WorkTableExec +11)--------------WorkTableExec: name=nodes # setup statement ok @@ -160,13 +163,15 @@ logical_plan physical_plan 01)SortExec: expr=[time@0 ASC NULLS LAST, name@1 ASC NULLS LAST, account_balance@2 ASC NULLS LAST], preserve_partitioning=[false] 02)--RecursiveQueryExec: name=balances, is_distinct=false -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], file_type=csv, 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 +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], file_type=csv, has_header=true +05)----CoalescePartitionsExec +06)------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] +07)--------CoalesceBatchesExec: target_batch_size=2 +08)----------FilterExec: time@0 < 10 +09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)--------------YieldStreamExec child=WorkTableExec +11)----------------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 @@ -731,16 +736,19 @@ logical_plan physical_plan 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 +03)----YieldStreamExec child=PlaceholderRowExec +04)------PlaceholderRowExec +05)--ProjectionExec: expr=[2 as val] +06)----CrossJoinExec +07)------CoalescePartitionsExec +08)--------CoalesceBatchesExec: target_batch_size=8182 +09)----------FilterExec: val@0 < 2 +10)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------------YieldStreamExec child=WorkTableExec +12)----------------WorkTableExec: name=recursive_cte +13)------ProjectionExec: expr=[2 as val] +14)--------YieldStreamExec child=PlaceholderRowExec +15)----------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/9794 # Non-recursive term and recursive term have different types @@ -872,7 +880,8 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query TT explain with numbers(a,b,c) as (select 1,2,3) select * from numbers; @@ -883,7 +892,8 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query TT explain with numbers as (select 1 as a, 2 as b, 3 as c) select * from numbers; @@ -894,7 +904,8 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec statement count 0 create table person (id int, name string, primary key(id)) @@ -911,8 +922,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8182 02)--HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(id@0, id@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; @@ -934,9 +947,11 @@ logical_plan 05)------EmptyRelation physical_plan 01)CrossJoinExec -02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--ProjectionExec: expr=[1 as Int64(1)] -04)----PlaceholderRowExec +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)--ProjectionExec: expr=[1 as Int64(1)] +05)----YieldStreamExec child=PlaceholderRowExec +06)------PlaceholderRowExec statement count 0 drop table j1; @@ -962,13 +977,15 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=numbers, is_distinct=false 02)--ProjectionExec: expr=[1 as n] -03)----PlaceholderRowExec -04)--CoalescePartitionsExec -05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] -06)------CoalesceBatchesExec: target_batch_size=8182 -07)--------FilterExec: n@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)------------WorkTableExec: name=numbers +03)----YieldStreamExec child=PlaceholderRowExec +04)------PlaceholderRowExec +05)--CoalescePartitionsExec +06)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] +07)------CoalesceBatchesExec: target_batch_size=8182 +08)--------FilterExec: n@0 < 10 +09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------YieldStreamExec child=WorkTableExec +11)--------------WorkTableExec: name=numbers query TT explain WITH RECURSIVE numbers AS ( @@ -988,13 +1005,15 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=numbers, is_distinct=false 02)--ProjectionExec: expr=[1 as n] -03)----PlaceholderRowExec -04)--CoalescePartitionsExec -05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] -06)------CoalesceBatchesExec: target_batch_size=8182 -07)--------FilterExec: n@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)------------WorkTableExec: name=numbers +03)----YieldStreamExec child=PlaceholderRowExec +04)------PlaceholderRowExec +05)--CoalescePartitionsExec +06)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] +07)------CoalesceBatchesExec: target_batch_size=8182 +08)--------FilterExec: n@0 < 10 +09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------YieldStreamExec child=WorkTableExec +11)--------------WorkTableExec: name=numbers statement count 0 set datafusion.execution.enable_recursive_ctes = false; diff --git a/datafusion/sqllogictest/test_files/ddl.slt b/datafusion/sqllogictest/test_files/ddl.slt index 1e95e426f3e0..674c463db4f1 100644 --- a/datafusion/sqllogictest/test_files/ddl.slt +++ b/datafusion/sqllogictest/test_files/ddl.slt @@ -713,7 +713,9 @@ query TT explain select c1 from t; ---- logical_plan TableScan: t projection=[c1] -physical_plan StreamingTableExec: partition_sizes=1, projection=[c1], infinite_source=true +physical_plan +01)YieldStreamExec child=StreamingTableExec +02)--StreamingTableExec: partition_sizes=1, projection=[c1], infinite_source=true statement ok drop table t; @@ -731,7 +733,9 @@ query TT explain select c1 from t; ---- logical_plan TableScan: t projection=[c1] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/empty.csv]]}, projection=[c1], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/empty.csv]]}, projection=[c1], file_type=csv, has_header=true statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt index d241e61f33ff..02be60626050 100644 --- a/datafusion/sqllogictest/test_files/dictionary.slt +++ b/datafusion/sqllogictest/test_files/dictionary.slt @@ -412,7 +412,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column2@1 = 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] # try literal = col to verify order doesn't matter # filter should not cast column2 @@ -425,7 +426,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column2@1 = 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] # Now query using an integer which must be coerced into a dictionary string @@ -443,7 +445,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column2@1 = 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] # Window Functions query I diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index b4a491619e89..167c4dd9cf96 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -102,7 +102,8 @@ physical_plan 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) 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]] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, 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 2df8a9dfbae4..d6e112422f42 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -46,7 +46,8 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: c2@1 > 10, projection=[c1@0] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], file_type=csv, has_header=true # explain_csv_exec_scan_config @@ -77,7 +78,9 @@ explain SELECT c1 FROM aggregate_test_100_with_order order by c1 ASC limit 10 logical_plan 01)Sort: aggregate_test_100_with_order.c1 ASC NULLS LAST, fetch=10 02)--TableScan: aggregate_test_100_with_order projection=[c1] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_test_100_order_by_c1_asc.csv]]}, projection=[c1], limit=10, output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_test_100_order_by_c1_asc.csv]]}, projection=[c1], limit=10, output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true ## explain_physical_plan_only @@ -89,7 +92,8 @@ EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c ---- physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec statement ok set datafusion.explain.physical_plan_only = false @@ -130,7 +134,9 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test ---- logical_plan TableScan: simple_explain_test projection=[a, b, c] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true # create a sink table, path is same with aggregate_test_100 table # we do not overwrite this file, we only assert plan. @@ -165,7 +171,8 @@ logical_plan physical_plan 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], preserve_partitioning=[false] -03)----DataSourceExec: 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], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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], file_type=csv, has_header=true # test EXPLAIN VERBOSE query TT @@ -242,10 +249,19 @@ physical_plan after OutputRequirements DataSourceExec: file_groups={1 group: [[W physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after wrap_leaves +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true -physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] -physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true +physical_plan_with_stats +01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +physical_plan_with_schema +01)YieldStreamExec child=DataSourceExec, schema=[a:Int32;N, b:Int32;N, c:Int32;N] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] ### tests for EXPLAIN with display schema enabled @@ -257,7 +273,9 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test; ---- logical_plan TableScan: simple_explain_test projection=[a, b, c] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] +physical_plan +01)YieldStreamExec child=DataSourceExec, schema=[a:Int32;N, b:Int32;N, c:Int32;N] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] statement ok @@ -275,7 +293,9 @@ set datafusion.explain.physical_plan_only = true; query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +physical_plan +01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] # Parquet scan with statistics collected statement ok @@ -287,7 +307,9 @@ CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-t query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- -physical_plan DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan +01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, 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)--DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(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 @@ -319,9 +341,16 @@ physical_plan after OutputRequirements physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(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 ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after wrap_leaves +01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, 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)--DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(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 SanityCheckPlan SAME TEXT AS ABOVE -physical_plan DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan_with_schema DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +physical_plan +01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, 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)--DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan_with_schema +01)YieldStreamExec child=DataSourceExec, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +02)--DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok @@ -360,10 +389,19 @@ physical_plan after OutputRequirements physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: 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, file_type=parquet physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after wrap_leaves +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: 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, file_type=parquet physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan DataSourceExec: 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, file_type=parquet -physical_plan_with_stats DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan_with_schema DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: 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, file_type=parquet +physical_plan_with_stats +01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, 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)--DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan_with_schema +01)YieldStreamExec child=DataSourceExec, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +02)--DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok @@ -382,7 +420,8 @@ logical_plan 02)--EmptyRelation physical_plan 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 +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query TT explain select [[1, 2, 3], [4, 5, 6]]; @@ -392,7 +431,8 @@ logical_plan 02)--EmptyRelation physical_plan 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 +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec # Explain Struct @@ -404,7 +444,8 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[{c0:1,c1:2.3,c2:abc} as struct(Int64(1),Float64(2.3),Utf8("abc"))] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec statement ok @@ -425,9 +466,11 @@ logical_plan 06)--------TableScan: t2 projection=[] physical_plan 01)NestedLoopJoinExec: join_type=LeftSemi -02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--ProjectionExec: expr=[] -04)----PlaceholderRowExec +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)--ProjectionExec: expr=[] +05)----YieldStreamExec child=PlaceholderRowExec +06)------PlaceholderRowExec statement ok drop table t1; @@ -445,14 +488,18 @@ query TT explain select * from values (1); ---- logical_plan Values: (Int64(1)) -physical_plan DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: partitions=1, partition_sizes=[1] # can explicitly request indent mode query TT explain format indent select * from values (1); ---- logical_plan Values: (Int64(1)) -physical_plan DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: partitions=1, partition_sizes=[1] # tree mode query TT @@ -460,12 +507,18 @@ explain format tree select * from values (1); ---- physical_plan 01)┌───────────────────────────┐ -02)│ DataSourceExec │ +02)│ yield_stream_exec │ 03)│ -------------------- │ -04)│ bytes: 128 │ -05)│ format: memory │ -06)│ rows: 1 │ -07)└───────────────────────────┘ +04)│ YieldStreamExec child: │ +05)│ DataSourceExec │ +06)└─────────────┬─────────────┘ +07)┌─────────────┴─────────────┐ +08)│ DataSourceExec │ +09)│ -------------------- │ +10)│ bytes: 128 │ +11)│ format: memory │ +12)│ rows: 1 │ +13)└───────────────────────────┘ # is not case sensitive query TT @@ -473,12 +526,18 @@ explain format TrEE select * from values (1); ---- physical_plan 01)┌───────────────────────────┐ -02)│ DataSourceExec │ +02)│ yield_stream_exec │ 03)│ -------------------- │ -04)│ bytes: 128 │ -05)│ format: memory │ -06)│ rows: 1 │ -07)└───────────────────────────┘ +04)│ YieldStreamExec child: │ +05)│ DataSourceExec │ +06)└─────────────┬─────────────┘ +07)┌─────────────┴─────────────┐ +08)│ DataSourceExec │ +09)│ -------------------- │ +10)│ bytes: 128 │ +11)│ format: memory │ +12)│ rows: 1 │ +13)└───────────────────────────┘ # wrapped in single quotes query TT @@ -486,12 +545,18 @@ explain format 'tree' select * from values (1); ---- physical_plan 01)┌───────────────────────────┐ -02)│ DataSourceExec │ +02)│ yield_stream_exec │ 03)│ -------------------- │ -04)│ bytes: 128 │ -05)│ format: memory │ -06)│ rows: 1 │ -07)└───────────────────────────┘ +04)│ YieldStreamExec child: │ +05)│ DataSourceExec │ +06)└─────────────┬─────────────┘ +07)┌─────────────┴─────────────┐ +08)│ DataSourceExec │ +09)│ -------------------- │ +10)│ bytes: 128 │ +11)│ format: memory │ +12)│ rows: 1 │ +13)└───────────────────────────┘ # wrapped in double quotes query TT @@ -499,12 +564,18 @@ explain format "tree" select * from values (1); ---- physical_plan 01)┌───────────────────────────┐ -02)│ DataSourceExec │ +02)│ yield_stream_exec │ 03)│ -------------------- │ -04)│ bytes: 128 │ -05)│ format: memory │ -06)│ rows: 1 │ -07)└───────────────────────────┘ +04)│ YieldStreamExec child: │ +05)│ DataSourceExec │ +06)└─────────────┬─────────────┘ +07)┌─────────────┴─────────────┐ +08)│ DataSourceExec │ +09)│ -------------------- │ +10)│ bytes: 128 │ +11)│ format: memory │ +12)│ rows: 1 │ +13)└───────────────────────────┘ # number is not a valid format query error DataFusion error: SQL error: ParserError\("Expected: an explain format such as TREE, found: 123 at Line: 1, Column: 16"\) diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 15bf61576571..bb30d538e2aa 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -187,11 +187,17 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ +23)│ yield_stream_exec │ 24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +25)│ YieldStreamExec child: │ +26)│ DataSourceExec │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ DataSourceExec │ +30)│ -------------------- │ +31)│ files: 1 │ +32)│ format: csv │ +33)└───────────────────────────┘ # Aggregate query TT @@ -243,11 +249,17 @@ physical_plan 43)│ RoundRobinBatch(4) │ 44)└─────────────┬─────────────┘ 45)┌─────────────┴─────────────┐ -46)│ DataSourceExec │ +46)│ yield_stream_exec │ 47)│ -------------------- │ -48)│ files: 1 │ -49)│ format: csv │ -50)└───────────────────────────┘ +48)│ YieldStreamExec child: │ +49)│ DataSourceExec │ +50)└─────────────┬─────────────┘ +51)┌─────────────┴─────────────┐ +52)│ DataSourceExec │ +53)│ -------------------- │ +54)│ files: 1 │ +55)│ format: csv │ +56)└───────────────────────────┘ # Limit @@ -262,11 +274,17 @@ physical_plan 05)│ skip: 3 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ DataSourceExec │ +08)│ yield_stream_exec │ 09)│ -------------------- │ -10)│ files: 1 │ -11)│ format: csv │ -12)└───────────────────────────┘ +10)│ YieldStreamExec child: │ +11)│ DataSourceExec │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ DataSourceExec │ +15)│ -------------------- │ +16)│ files: 1 │ +17)│ format: csv │ +18)└───────────────────────────┘ query TT explain SELECT * FROM limit_table LIMIT 10; @@ -278,12 +296,18 @@ physical_plan 04)│ limit: 10 │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ DataSourceExec │ +07)│ yield_stream_exec │ 08)│ -------------------- │ -09)│ bytes: 3120 │ -10)│ format: memory │ -11)│ rows: 2 │ -12)└───────────────────────────┘ +09)│ YieldStreamExec child: │ +10)│ DataSourceExec │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ DataSourceExec │ +14)│ -------------------- │ +15)│ bytes: 3120 │ +16)│ format: memory │ +17)│ rows: 2 │ +18)└───────────────────────────┘ # 2 Joins query TT @@ -327,11 +351,17 @@ physical_plan 35)│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ 36)└─────────────┬─────────────┘└─────────────┬─────────────┘ 37)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -38)│ DataSourceExec ││ DataSourceExec │ +38)│ yield_stream_exec ││ yield_stream_exec │ 39)│ -------------------- ││ -------------------- │ -40)│ files: 1 ││ files: 1 │ -41)│ format: csv ││ format: parquet │ -42)└───────────────────────────┘└───────────────────────────┘ +40)│ YieldStreamExec child: ││ YieldStreamExec child: │ +41)│ DataSourceExec ││ DataSourceExec │ +42)└─────────────┬─────────────┘└─────────────┬─────────────┘ +43)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +44)│ DataSourceExec ││ DataSourceExec │ +45)│ -------------------- ││ -------------------- │ +46)│ files: 1 ││ files: 1 │ +47)│ format: csv ││ format: parquet │ +48)└───────────────────────────┘└───────────────────────────┘ # 3 Joins query TT @@ -365,18 +395,18 @@ physical_plan 19)│ (int_col = int_col) │ │ 20)└─────────────┬─────────────┘ │ 21)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -22)│ DataSourceExec ││ CoalesceBatchesExec │ +22)│ yield_stream_exec ││ CoalesceBatchesExec │ 23)│ -------------------- ││ -------------------- │ -24)│ bytes: 1560 ││ target_batch_size: │ -25)│ format: memory ││ 8192 │ -26)│ rows: 1 ││ │ -27)└───────────────────────────┘└─────────────┬─────────────┘ -28)-----------------------------┌─────────────┴─────────────┐ -29)-----------------------------│ HashJoinExec │ -30)-----------------------------│ -------------------- │ -31)-----------------------------│ on: ├──────────────┐ -32)-----------------------------│ (int_col = int_col) │ │ -33)-----------------------------└─────────────┬─────────────┘ │ +24)│ YieldStreamExec child: ││ target_batch_size: │ +25)│ DataSourceExec ││ 8192 │ +26)└─────────────┬─────────────┘└─────────────┬─────────────┘ +27)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +28)│ DataSourceExec ││ HashJoinExec │ +29)│ -------------------- ││ -------------------- │ +30)│ bytes: 1560 ││ on: ├──────────────┐ +31)│ format: memory ││ (int_col = int_col) │ │ +32)│ rows: 1 ││ │ │ +33)└───────────────────────────┘└─────────────┬─────────────┘ │ 34)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 35)-----------------------------│ CoalesceBatchesExec ││ CoalesceBatchesExec │ 36)-----------------------------│ -------------------- ││ -------------------- │ @@ -402,11 +432,17 @@ physical_plan 56)-----------------------------│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ 57)-----------------------------└─────────────┬─────────────┘└─────────────┬─────────────┘ 58)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -59)-----------------------------│ DataSourceExec ││ DataSourceExec │ +59)-----------------------------│ yield_stream_exec ││ yield_stream_exec │ 60)-----------------------------│ -------------------- ││ -------------------- │ -61)-----------------------------│ files: 1 ││ files: 1 │ -62)-----------------------------│ format: csv ││ format: parquet │ -63)-----------------------------└───────────────────────────┘└───────────────────────────┘ +61)-----------------------------│ YieldStreamExec child: ││ YieldStreamExec child: │ +62)-----------------------------│ DataSourceExec ││ DataSourceExec │ +63)-----------------------------└─────────────┬─────────────┘└─────────────┬─────────────┘ +64)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +65)-----------------------------│ DataSourceExec ││ DataSourceExec │ +66)-----------------------------│ -------------------- ││ -------------------- │ +67)-----------------------------│ files: 1 ││ files: 1 │ +68)-----------------------------│ format: csv ││ format: parquet │ +69)-----------------------------└───────────────────────────┘└───────────────────────────┘ # Long Filter (demonstrate what happens with wrapping) query TT @@ -441,11 +477,17 @@ physical_plan 24)│ RoundRobinBatch(4) │ 25)└─────────────┬─────────────┘ 26)┌─────────────┴─────────────┐ -27)│ DataSourceExec │ +27)│ yield_stream_exec │ 28)│ -------------------- │ -29)│ files: 1 │ -30)│ format: csv │ -31)└───────────────────────────┘ +29)│ YieldStreamExec child: │ +30)│ DataSourceExec │ +31)└─────────────┬─────────────┘ +32)┌─────────────┴─────────────┐ +33)│ DataSourceExec │ +34)│ -------------------- │ +35)│ files: 1 │ +36)│ format: csv │ +37)└───────────────────────────┘ # Check maximum line limit. query TT @@ -503,11 +545,17 @@ physical_plan 48)│ RoundRobinBatch(4) │ 49)└─────────────┬─────────────┘ 50)┌─────────────┴─────────────┐ -51)│ DataSourceExec │ +51)│ yield_stream_exec │ 52)│ -------------------- │ -53)│ files: 1 │ -54)│ format: csv │ -55)└───────────────────────────┘ +53)│ YieldStreamExec child: │ +54)│ DataSourceExec │ +55)└─────────────┬─────────────┘ +56)┌─────────────┴─────────────┐ +57)│ DataSourceExec │ +58)│ -------------------- │ +59)│ files: 1 │ +60)│ format: csv │ +61)└───────────────────────────┘ # Check exactly the render width. query TT @@ -537,11 +585,17 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ +23)│ yield_stream_exec │ 24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +25)│ YieldStreamExec child: │ +26)│ DataSourceExec │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ DataSourceExec │ +30)│ -------------------- │ +31)│ files: 1 │ +32)│ format: csv │ +33)└───────────────────────────┘ # Check with the render witdth + 1. query TT @@ -573,11 +627,17 @@ physical_plan 22)│ RoundRobinBatch(4) │ 23)└─────────────┬─────────────┘ 24)┌─────────────┴─────────────┐ -25)│ DataSourceExec │ +25)│ yield_stream_exec │ 26)│ -------------------- │ -27)│ files: 1 │ -28)│ format: csv │ -29)└───────────────────────────┘ +27)│ YieldStreamExec child: │ +28)│ DataSourceExec │ +29)└─────────────┬─────────────┘ +30)┌─────────────┴─────────────┐ +31)│ DataSourceExec │ +32)│ -------------------- │ +33)│ files: 1 │ +34)│ format: csv │ +35)└───────────────────────────┘ # Query with filter on csv query TT @@ -606,11 +666,17 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ +23)│ yield_stream_exec │ 24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +25)│ YieldStreamExec child: │ +26)│ DataSourceExec │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ DataSourceExec │ +30)│ -------------------- │ +31)│ files: 1 │ +32)│ format: csv │ +33)└───────────────────────────┘ # Query with filter on parquet @@ -640,14 +706,20 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ +23)│ yield_stream_exec │ 24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: parquet │ -27)│ │ -28)│ predicate: │ -29)│ string_col != foo │ -30)└───────────────────────────┘ +25)│ YieldStreamExec child: │ +26)│ DataSourceExec │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ DataSourceExec │ +30)│ -------------------- │ +31)│ files: 1 │ +32)│ format: parquet │ +33)│ │ +34)│ predicate: │ +35)│ string_col != foo │ +36)└───────────────────────────┘ # Query with filter on memory query TT @@ -667,12 +739,18 @@ physical_plan 11)│ string_col != foo │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ DataSourceExec │ +14)│ yield_stream_exec │ 15)│ -------------------- │ -16)│ bytes: 1560 │ -17)│ format: memory │ -18)│ rows: 1 │ -19)└───────────────────────────┘ +16)│ YieldStreamExec child: │ +17)│ DataSourceExec │ +18)└─────────────┬─────────────┘ +19)┌─────────────┴─────────────┐ +20)│ DataSourceExec │ +21)│ -------------------- │ +22)│ bytes: 1560 │ +23)│ format: memory │ +24)│ rows: 1 │ +25)└───────────────────────────┘ # Query with filter on json query TT @@ -701,11 +779,17 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ +23)│ yield_stream_exec │ 24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: json │ -27)└───────────────────────────┘ +25)│ YieldStreamExec child: │ +26)│ DataSourceExec │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ DataSourceExec │ +30)│ -------------------- │ +31)│ files: 1 │ +32)│ format: json │ +33)└───────────────────────────┘ # Query with filter on arrow query TT @@ -734,11 +818,17 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ +23)│ yield_stream_exec │ 24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: arrow │ -27)└───────────────────────────┘ +25)│ YieldStreamExec child: │ +26)│ DataSourceExec │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ DataSourceExec │ +30)│ -------------------- │ +31)│ files: 1 │ +32)│ format: arrow │ +33)└───────────────────────────┘ # Query with window agg. @@ -767,11 +857,17 @@ physical_plan 19)│ FOLLOWING │ 20)└─────────────┬─────────────┘ 21)┌─────────────┴─────────────┐ -22)│ DataSourceExec │ +22)│ yield_stream_exec │ 23)│ -------------------- │ -24)│ files: 1 │ -25)│ format: csv │ -26)└───────────────────────────┘ +24)│ YieldStreamExec child: │ +25)│ DataSourceExec │ +26)└─────────────┬─────────────┘ +27)┌─────────────┴─────────────┐ +28)│ DataSourceExec │ +29)│ -------------------- │ +30)│ files: 1 │ +31)│ format: csv │ +32)└───────────────────────────┘ # Query with bounded window agg. query TT @@ -814,13 +910,19 @@ physical_plan 31)│ v1: value │ 32)└─────────────┬─────────────┘ 33)┌─────────────┴─────────────┐ -34)│ LazyMemoryExec │ +34)│ yield_stream_exec │ 35)│ -------------------- │ -36)│ batch_generators: │ -37)│ generate_series: start=1, │ -38)│ end=1000, batch_size │ -39)│ =8192 │ -40)└───────────────────────────┘ +36)│ YieldStreamExec child: │ +37)│ LazyMemoryExec │ +38)└─────────────┬─────────────┘ +39)┌─────────────┴─────────────┐ +40)│ LazyMemoryExec │ +41)│ -------------------- │ +42)│ batch_generators: │ +43)│ generate_series: start=1, │ +44)│ end=1000, batch_size │ +45)│ =8192 │ +46)└───────────────────────────┘ query TT explain select @@ -860,11 +962,17 @@ physical_plan 29)│ FOLLOWING │ 30)└─────────────┬─────────────┘ 31)┌─────────────┴─────────────┐ -32)│ DataSourceExec │ +32)│ yield_stream_exec │ 33)│ -------------------- │ -34)│ files: 1 │ -35)│ format: csv │ -36)└───────────────────────────┘ +34)│ YieldStreamExec child: │ +35)│ DataSourceExec │ +36)└─────────────┬─────────────┘ +37)┌─────────────┴─────────────┐ +38)│ DataSourceExec │ +39)│ -------------------- │ +40)│ files: 1 │ +41)│ format: csv │ +42)└───────────────────────────┘ # Query for sort. query TT @@ -877,11 +985,17 @@ physical_plan 04)│string_col@1 ASC NULLS LAST│ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ DataSourceExec │ +07)│ yield_stream_exec │ 08)│ -------------------- │ -09)│ files: 1 │ -10)│ format: csv │ -11)└───────────────────────────┘ +09)│ YieldStreamExec child: │ +10)│ DataSourceExec │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ DataSourceExec │ +14)│ -------------------- │ +15)│ files: 1 │ +16)│ format: csv │ +17)└───────────────────────────┘ # Query for sort with limit. query TT @@ -896,11 +1010,17 @@ physical_plan 06)│string_col@1 ASC NULLS LAST│ 07)└─────────────┬─────────────┘ 08)┌─────────────┴─────────────┐ -09)│ DataSourceExec │ +09)│ yield_stream_exec │ 10)│ -------------------- │ -11)│ files: 1 │ -12)│ format: csv │ -13)└───────────────────────────┘ +11)│ YieldStreamExec child: │ +12)│ DataSourceExec │ +13)└─────────────┬─────────────┘ +14)┌─────────────┴─────────────┐ +15)│ DataSourceExec │ +16)│ -------------------- │ +17)│ files: 1 │ +18)│ format: csv │ +19)└───────────────────────────┘ # Query with projection on csv query TT @@ -929,11 +1049,17 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ +23)│ yield_stream_exec │ 24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +25)│ YieldStreamExec child: │ +26)│ DataSourceExec │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ DataSourceExec │ +30)│ -------------------- │ +31)│ files: 1 │ +32)│ format: csv │ +33)└───────────────────────────┘ query TT explain select @@ -1005,11 +1131,17 @@ physical_plan 61)│ int_col@0 DESC │ 62)└─────────────┬─────────────┘ 63)┌─────────────┴─────────────┐ -64)│ DataSourceExec │ +64)│ yield_stream_exec │ 65)│ -------------------- │ -66)│ files: 1 │ -67)│ format: csv │ -68)└───────────────────────────┘ +66)│ YieldStreamExec child: │ +67)│ DataSourceExec │ +68)└─────────────┬─────────────┘ +69)┌─────────────┴─────────────┐ +70)│ DataSourceExec │ +71)│ -------------------- │ +72)│ files: 1 │ +73)│ format: csv │ +74)└───────────────────────────┘ # Query with projection on parquet query TT @@ -1038,11 +1170,17 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ +23)│ yield_stream_exec │ 24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: parquet │ -27)└───────────────────────────┘ +25)│ YieldStreamExec child: │ +26)│ DataSourceExec │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ DataSourceExec │ +30)│ -------------------- │ +31)│ files: 1 │ +32)│ format: parquet │ +33)└───────────────────────────┘ # Query with projection on memory @@ -1063,12 +1201,18 @@ physical_plan 11)│ bigint_col │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ DataSourceExec │ +14)│ yield_stream_exec │ 15)│ -------------------- │ -16)│ bytes: 1560 │ -17)│ format: memory │ -18)│ rows: 1 │ -19)└───────────────────────────┘ +16)│ YieldStreamExec child: │ +17)│ DataSourceExec │ +18)└─────────────┬─────────────┘ +19)┌─────────────┴─────────────┐ +20)│ DataSourceExec │ +21)│ -------------------- │ +22)│ bytes: 1560 │ +23)│ format: memory │ +24)│ rows: 1 │ +25)└───────────────────────────┘ # Query with projection on json query TT @@ -1096,11 +1240,17 @@ physical_plan 19)│ RoundRobinBatch(4) │ 20)└─────────────┬─────────────┘ 21)┌─────────────┴─────────────┐ -22)│ DataSourceExec │ +22)│ yield_stream_exec │ 23)│ -------------------- │ -24)│ files: 1 │ -25)│ format: json │ -26)└───────────────────────────┘ +24)│ YieldStreamExec child: │ +25)│ DataSourceExec │ +26)└─────────────┬─────────────┘ +27)┌─────────────┴─────────────┐ +28)│ DataSourceExec │ +29)│ -------------------- │ +30)│ files: 1 │ +31)│ format: json │ +32)└───────────────────────────┘ # Query with projection on arrow @@ -1130,11 +1280,17 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ +23)│ yield_stream_exec │ 24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: arrow │ -27)└───────────────────────────┘ +25)│ YieldStreamExec child: │ +26)│ DataSourceExec │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ DataSourceExec │ +30)│ -------------------- │ +31)│ files: 1 │ +32)│ format: arrow │ +33)└───────────────────────────┘ # Query with PartialSortExec. query TT @@ -1151,11 +1307,17 @@ physical_plan 06)│ ASC NULLS LAST │ 07)└─────────────┬─────────────┘ 08)┌─────────────┴─────────────┐ -09)│ StreamingTableExec │ +09)│ yield_stream_exec │ 10)│ -------------------- │ -11)│ infinite: true │ -12)│ limit: None │ -13)└───────────────────────────┘ +11)│ YieldStreamExec child: │ +12)│ StreamingTableExec │ +13)└─────────────┬─────────────┘ +14)┌─────────────┴─────────────┐ +15)│ StreamingTableExec │ +16)│ -------------------- │ +17)│ infinite: true │ +18)│ limit: None │ +19)└───────────────────────────┘ query TT EXPLAIN SELECT * @@ -1174,11 +1336,17 @@ physical_plan 08)│ limit: 50 │ 09)└─────────────┬─────────────┘ 10)┌─────────────┴─────────────┐ -11)│ StreamingTableExec │ +11)│ yield_stream_exec │ 12)│ -------------------- │ -13)│ infinite: true │ -14)│ limit: None │ -15)└───────────────────────────┘ +13)│ YieldStreamExec child: │ +14)│ StreamingTableExec │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ StreamingTableExec │ +18)│ -------------------- │ +19)│ infinite: true │ +20)│ limit: None │ +21)└───────────────────────────┘ # Query with hash join. query TT @@ -1235,20 +1403,26 @@ physical_plan 48)│ string_col ││ │ 49)└─────────────┬─────────────┘└─────────────┬─────────────┘ 50)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -51)│ RepartitionExec ││ DataSourceExec │ +51)│ RepartitionExec ││ yield_stream_exec │ 52)│ -------------------- ││ -------------------- │ -53)│ partition_count(in->out): ││ files: 1 │ -54)│ 1 -> 4 ││ format: parquet │ +53)│ partition_count(in->out): ││ YieldStreamExec child: │ +54)│ 1 -> 4 ││ DataSourceExec │ 55)│ ││ │ 56)│ partitioning_scheme: ││ │ 57)│ RoundRobinBatch(4) ││ │ -58)└─────────────┬─────────────┘└───────────────────────────┘ -59)┌─────────────┴─────────────┐ -60)│ DataSourceExec │ -61)│ -------------------- │ -62)│ files: 1 │ -63)│ format: csv │ -64)└───────────────────────────┘ +58)└─────────────┬─────────────┘└─────────────┬─────────────┘ +59)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +60)│ yield_stream_exec ││ DataSourceExec │ +61)│ -------------------- ││ -------------------- │ +62)│ YieldStreamExec child: ││ files: 1 │ +63)│ DataSourceExec ││ format: parquet │ +64)└─────────────┬─────────────┘└───────────────────────────┘ +65)┌─────────────┴─────────────┐ +66)│ DataSourceExec │ +67)│ -------------------- │ +68)│ files: 1 │ +69)│ format: csv │ +70)└───────────────────────────┘ # Query with outer hash join. query TT @@ -1307,20 +1481,26 @@ physical_plan 50)│ string_col ││ │ 51)└─────────────┬─────────────┘└─────────────┬─────────────┘ 52)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -53)│ RepartitionExec ││ DataSourceExec │ +53)│ RepartitionExec ││ yield_stream_exec │ 54)│ -------------------- ││ -------------------- │ -55)│ partition_count(in->out): ││ files: 1 │ -56)│ 1 -> 4 ││ format: parquet │ +55)│ partition_count(in->out): ││ YieldStreamExec child: │ +56)│ 1 -> 4 ││ DataSourceExec │ 57)│ ││ │ 58)│ partitioning_scheme: ││ │ 59)│ RoundRobinBatch(4) ││ │ -60)└─────────────┬─────────────┘└───────────────────────────┘ -61)┌─────────────┴─────────────┐ -62)│ DataSourceExec │ -63)│ -------------------- │ -64)│ files: 1 │ -65)│ format: csv │ -66)└───────────────────────────┘ +60)└─────────────┬─────────────┘└─────────────┬─────────────┘ +61)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +62)│ yield_stream_exec ││ DataSourceExec │ +63)│ -------------------- ││ -------------------- │ +64)│ YieldStreamExec child: ││ files: 1 │ +65)│ DataSourceExec ││ format: parquet │ +66)└─────────────┬─────────────┘└───────────────────────────┘ +67)┌─────────────┴─────────────┐ +68)│ DataSourceExec │ +69)│ -------------------- │ +70)│ files: 1 │ +71)│ format: csv │ +72)└───────────────────────────┘ # Query with nested loop join. query TT @@ -1333,17 +1513,17 @@ physical_plan 04)│ join_type: LeftSemi │ │ 05)└─────────────┬─────────────┘ │ 06)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -07)│ DataSourceExec ││ ProjectionExec │ +07)│ yield_stream_exec ││ ProjectionExec │ 08)│ -------------------- ││ │ -09)│ files: 1 ││ │ -10)│ format: csv ││ │ -11)└───────────────────────────┘└─────────────┬─────────────┘ -12)-----------------------------┌─────────────┴─────────────┐ -13)-----------------------------│ AggregateExec │ -14)-----------------------------│ -------------------- │ -15)-----------------------------│ aggr: count(1) │ -16)-----------------------------│ mode: Final │ -17)-----------------------------└─────────────┬─────────────┘ +09)│ YieldStreamExec child: ││ │ +10)│ DataSourceExec ││ │ +11)└─────────────┬─────────────┘└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +13)│ DataSourceExec ││ AggregateExec │ +14)│ -------------------- ││ -------------------- │ +15)│ files: 1 ││ aggr: count(1) │ +16)│ format: csv ││ mode: Final │ +17)└───────────────────────────┘└─────────────┬─────────────┘ 18)-----------------------------┌─────────────┴─────────────┐ 19)-----------------------------│ CoalescePartitionsExec │ 20)-----------------------------└─────────────┬─────────────┘ @@ -1363,11 +1543,17 @@ physical_plan 34)-----------------------------│ RoundRobinBatch(4) │ 35)-----------------------------└─────────────┬─────────────┘ 36)-----------------------------┌─────────────┴─────────────┐ -37)-----------------------------│ DataSourceExec │ +37)-----------------------------│ yield_stream_exec │ 38)-----------------------------│ -------------------- │ -39)-----------------------------│ files: 1 │ -40)-----------------------------│ format: parquet │ -41)-----------------------------└───────────────────────────┘ +39)-----------------------------│ YieldStreamExec child: │ +40)-----------------------------│ DataSourceExec │ +41)-----------------------------└─────────────┬─────────────┘ +42)-----------------------------┌─────────────┴─────────────┐ +43)-----------------------------│ DataSourceExec │ +44)-----------------------------│ -------------------- │ +45)-----------------------------│ files: 1 │ +46)-----------------------------│ format: parquet │ +47)-----------------------------└───────────────────────────┘ # Query with cross join. query TT @@ -1378,20 +1564,26 @@ physical_plan 02)│ CrossJoinExec ├──────────────┐ 03)└─────────────┬─────────────┘ │ 04)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -05)│ DataSourceExec ││ RepartitionExec │ +05)│ yield_stream_exec ││ RepartitionExec │ 06)│ -------------------- ││ -------------------- │ -07)│ files: 1 ││ partition_count(in->out): │ -08)│ format: csv ││ 1 -> 4 │ +07)│ YieldStreamExec child: ││ partition_count(in->out): │ +08)│ DataSourceExec ││ 1 -> 4 │ 09)│ ││ │ 10)│ ││ partitioning_scheme: │ 11)│ ││ RoundRobinBatch(4) │ -12)└───────────────────────────┘└─────────────┬─────────────┘ -13)-----------------------------┌─────────────┴─────────────┐ -14)-----------------------------│ DataSourceExec │ -15)-----------------------------│ -------------------- │ -16)-----------------------------│ files: 1 │ -17)-----------------------------│ format: parquet │ -18)-----------------------------└───────────────────────────┘ +12)└─────────────┬─────────────┘└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +14)│ DataSourceExec ││ yield_stream_exec │ +15)│ -------------------- ││ -------------------- │ +16)│ files: 1 ││ YieldStreamExec child: │ +17)│ format: csv ││ DataSourceExec │ +18)└───────────────────────────┘└─────────────┬─────────────┘ +19)-----------------------------┌─────────────┴─────────────┐ +20)-----------------------------│ DataSourceExec │ +21)-----------------------------│ -------------------- │ +22)-----------------------------│ files: 1 │ +23)-----------------------------│ format: parquet │ +24)-----------------------------└───────────────────────────┘ # Query with sort merge join. @@ -1413,12 +1605,18 @@ physical_plan 09)│ c1@0 ASC ││ c1@0 ASC │ 10)└─────────────┬─────────────┘└─────────────┬─────────────┘ 11)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -12)│ DataSourceExec ││ DataSourceExec │ +12)│ yield_stream_exec ││ yield_stream_exec │ 13)│ -------------------- ││ -------------------- │ -14)│ bytes: 6040 ││ bytes: 6040 │ -15)│ format: memory ││ format: memory │ -16)│ rows: 1 ││ rows: 1 │ -17)└───────────────────────────┘└───────────────────────────┘ +14)│ YieldStreamExec child: ││ YieldStreamExec child: │ +15)│ DataSourceExec ││ DataSourceExec │ +16)└─────────────┬─────────────┘└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +18)│ DataSourceExec ││ DataSourceExec │ +19)│ -------------------- ││ -------------------- │ +20)│ bytes: 6040 ││ bytes: 6040 │ +21)│ format: memory ││ format: memory │ +22)│ rows: 1 ││ rows: 1 │ +23)└───────────────────────────┘└───────────────────────────┘ statement ok set datafusion.optimizer.prefer_hash_join = true; @@ -1527,12 +1725,18 @@ physical_plan 55)│ mode: Partial ││ mode: Partial │ 56)└─────────────┬─────────────┘└─────────────┬─────────────┘ 57)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -58)│ DataSourceExec ││ DataSourceExec │ +58)│ yield_stream_exec ││ yield_stream_exec │ 59)│ -------------------- ││ -------------------- │ -60)│ bytes: 1320 ││ bytes: 1312 │ -61)│ format: memory ││ format: memory │ -62)│ rows: 1 ││ rows: 1 │ -63)└───────────────────────────┘└───────────────────────────┘ +60)│ YieldStreamExec child: ││ YieldStreamExec child: │ +61)│ DataSourceExec ││ DataSourceExec │ +62)└─────────────┬─────────────┘└─────────────┬─────────────┘ +63)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +64)│ DataSourceExec ││ DataSourceExec │ +65)│ -------------------- ││ -------------------- │ +66)│ bytes: 1320 ││ bytes: 1312 │ +67)│ format: memory ││ format: memory │ +68)│ rows: 1 ││ rows: 1 │ +69)└───────────────────────────┘└───────────────────────────┘ # Test explain tree for UnionExec query TT @@ -1546,19 +1750,25 @@ physical_plan 02)│ UnionExec ├──────────────┐ 03)└─────────────┬─────────────┘ │ 04)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -05)│ DataSourceExec ││ ProjectionExec │ +05)│ yield_stream_exec ││ ProjectionExec │ 06)│ -------------------- ││ -------------------- │ -07)│ bytes: 1320 ││ id: CAST(id AS Int32) │ -08)│ format: memory ││ name: name │ -09)│ rows: 1 ││ │ -10)└───────────────────────────┘└─────────────┬─────────────┘ -11)-----------------------------┌─────────────┴─────────────┐ -12)-----------------------------│ DataSourceExec │ -13)-----------------------------│ -------------------- │ -14)-----------------------------│ bytes: 1312 │ -15)-----------------------------│ format: memory │ -16)-----------------------------│ rows: 1 │ -17)-----------------------------└───────────────────────────┘ +07)│ YieldStreamExec child: ││ id: CAST(id AS Int32) │ +08)│ DataSourceExec ││ name: name │ +09)└─────────────┬─────────────┘└─────────────┬─────────────┘ +10)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +11)│ DataSourceExec ││ yield_stream_exec │ +12)│ -------------------- ││ -------------------- │ +13)│ bytes: 1320 ││ YieldStreamExec child: │ +14)│ format: memory ││ DataSourceExec │ +15)│ rows: 1 ││ │ +16)└───────────────────────────┘└─────────────┬─────────────┘ +17)-----------------------------┌─────────────┴─────────────┐ +18)-----------------------------│ DataSourceExec │ +19)-----------------------------│ -------------------- │ +20)-----------------------------│ bytes: 1312 │ +21)-----------------------------│ format: memory │ +22)-----------------------------│ rows: 1 │ +23)-----------------------------└───────────────────────────┘ # cleanup statement ok @@ -1613,11 +1823,17 @@ physical_plan 25)│ RoundRobinBatch(4) │ 26)└─────────────┬─────────────┘ 27)┌─────────────┴─────────────┐ -28)│ StreamingTableExec │ +28)│ yield_stream_exec │ 29)│ -------------------- │ -30)│ infinite: true │ -31)│ limit: None │ -32)└───────────────────────────┘ +30)│ YieldStreamExec child: │ +31)│ StreamingTableExec │ +32)└─────────────┬─────────────┘ +33)┌─────────────┴─────────────┐ +34)│ StreamingTableExec │ +35)│ -------------------- │ +36)│ infinite: true │ +37)│ limit: None │ +38)└───────────────────────────┘ # constant ticker, CAST(time AS DATE) = time, order by time @@ -1655,11 +1871,17 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ +29)│ yield_stream_exec │ 30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +31)│ YieldStreamExec child: │ +32)│ StreamingTableExec │ +33)└─────────────┬─────────────┘ +34)┌─────────────┴─────────────┐ +35)│ StreamingTableExec │ +36)│ -------------------- │ +37)│ infinite: true │ +38)│ limit: None │ +39)└───────────────────────────┘ # same thing but order by date query TT @@ -1696,11 +1918,17 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ +29)│ yield_stream_exec │ 30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +31)│ YieldStreamExec child: │ +32)│ StreamingTableExec │ +33)└─────────────┬─────────────┘ +34)┌─────────────┴─────────────┐ +35)│ StreamingTableExec │ +36)│ -------------------- │ +37)│ infinite: true │ +38)│ limit: None │ +39)└───────────────────────────┘ # same thing but order by ticker query TT @@ -1735,11 +1963,17 @@ physical_plan 24)│ RoundRobinBatch(4) │ 25)└─────────────┬─────────────┘ 26)┌─────────────┴─────────────┐ -27)│ StreamingTableExec │ +27)│ yield_stream_exec │ 28)│ -------------------- │ -29)│ infinite: true │ -30)│ limit: None │ -31)└───────────────────────────┘ +29)│ YieldStreamExec child: │ +30)│ StreamingTableExec │ +31)└─────────────┬─────────────┘ +32)┌─────────────┴─────────────┐ +33)│ StreamingTableExec │ +34)│ -------------------- │ +35)│ infinite: true │ +36)│ limit: None │ +37)└───────────────────────────┘ # same thing but order by time, date @@ -1778,11 +2012,17 @@ physical_plan 27)│ RoundRobinBatch(4) │ 28)└─────────────┬─────────────┘ 29)┌─────────────┴─────────────┐ -30)│ StreamingTableExec │ +30)│ yield_stream_exec │ 31)│ -------------------- │ -32)│ infinite: true │ -33)│ limit: None │ -34)└───────────────────────────┘ +32)│ YieldStreamExec child: │ +33)│ StreamingTableExec │ +34)└─────────────┬─────────────┘ +35)┌─────────────┴─────────────┐ +36)│ StreamingTableExec │ +37)│ -------------------- │ +38)│ infinite: true │ +39)│ limit: None │ +40)└───────────────────────────┘ @@ -1822,11 +2062,17 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ +29)│ yield_stream_exec │ 30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +31)│ YieldStreamExec child: │ +32)│ StreamingTableExec │ +33)└─────────────┬─────────────┘ +34)┌─────────────┴─────────────┐ +35)│ StreamingTableExec │ +36)│ -------------------- │ +37)│ infinite: true │ +38)│ limit: None │ +39)└───────────────────────────┘ @@ -1851,35 +2097,42 @@ physical_plan 07)│ id: 1 ││ │ 08)└─────────────┬─────────────┘└─────────────┬─────────────┘ 09)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -10)│ PlaceholderRowExec ││ ProjectionExec │ -11)│ ││ -------------------- │ -12)│ ││ id: id + 1 │ -13)└───────────────────────────┘└─────────────┬─────────────┘ -14)-----------------------------┌─────────────┴─────────────┐ -15)-----------------------------│ CoalesceBatchesExec │ -16)-----------------------------│ -------------------- │ -17)-----------------------------│ target_batch_size: │ -18)-----------------------------│ 8192 │ -19)-----------------------------└─────────────┬─────────────┘ -20)-----------------------------┌─────────────┴─────────────┐ -21)-----------------------------│ FilterExec │ -22)-----------------------------│ -------------------- │ -23)-----------------------------│ predicate: id < 10 │ -24)-----------------------------└─────────────┬─────────────┘ -25)-----------------------------┌─────────────┴─────────────┐ -26)-----------------------------│ RepartitionExec │ -27)-----------------------------│ -------------------- │ -28)-----------------------------│ partition_count(in->out): │ -29)-----------------------------│ 1 -> 4 │ -30)-----------------------------│ │ -31)-----------------------------│ partitioning_scheme: │ -32)-----------------------------│ RoundRobinBatch(4) │ -33)-----------------------------└─────────────┬─────────────┘ -34)-----------------------------┌─────────────┴─────────────┐ -35)-----------------------------│ WorkTableExec │ -36)-----------------------------│ -------------------- │ -37)-----------------------------│ name: nodes │ -38)-----------------------------└───────────────────────────┘ +10)│ yield_stream_exec ││ ProjectionExec │ +11)│ -------------------- ││ -------------------- │ +12)│ YieldStreamExec child: ││ id: id + 1 │ +13)│ PlaceholderRowExec ││ │ +14)└─────────────┬─────────────┘└─────────────┬─────────────┘ +15)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +16)│ PlaceholderRowExec ││ CoalesceBatchesExec │ +17)│ ││ -------------------- │ +18)│ ││ target_batch_size: │ +19)│ ││ 8192 │ +20)└───────────────────────────┘└─────────────┬─────────────┘ +21)-----------------------------┌─────────────┴─────────────┐ +22)-----------------------------│ FilterExec │ +23)-----------------------------│ -------------------- │ +24)-----------------------------│ predicate: id < 10 │ +25)-----------------------------└─────────────┬─────────────┘ +26)-----------------------------┌─────────────┴─────────────┐ +27)-----------------------------│ RepartitionExec │ +28)-----------------------------│ -------------------- │ +29)-----------------------------│ partition_count(in->out): │ +30)-----------------------------│ 1 -> 4 │ +31)-----------------------------│ │ +32)-----------------------------│ partitioning_scheme: │ +33)-----------------------------│ RoundRobinBatch(4) │ +34)-----------------------------└─────────────┬─────────────┘ +35)-----------------------------┌─────────────┴─────────────┐ +36)-----------------------------│ yield_stream_exec │ +37)-----------------------------│ -------------------- │ +38)-----------------------------│ YieldStreamExec child: │ +39)-----------------------------│ WorkTableExec │ +40)-----------------------------└─────────────┬─────────────┘ +41)-----------------------------┌─────────────┴─────────────┐ +42)-----------------------------│ WorkTableExec │ +43)-----------------------------│ -------------------- │ +44)-----------------------------│ name: nodes │ +45)-----------------------------└───────────────────────────┘ query TT explain COPY (VALUES (1, 'foo', 1, '2023-01-01'), (2, 'bar', 2, '2023-01-02'), (3, 'baz', 3, '2023-01-03')) @@ -1897,12 +2150,18 @@ physical_plan 09)│ format: json │ 10)└─────────────┬─────────────┘ 11)┌─────────────┴─────────────┐ -12)│ DataSourceExec │ +12)│ yield_stream_exec │ 13)│ -------------------- │ -14)│ bytes: 2672 │ -15)│ format: memory │ -16)│ rows: 1 │ -17)└───────────────────────────┘ +14)│ YieldStreamExec child: │ +15)│ DataSourceExec │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ DataSourceExec │ +19)│ -------------------- │ +20)│ bytes: 2672 │ +21)│ format: memory │ +22)│ rows: 1 │ +23)└───────────────────────────┘ query TT explain COPY (VALUES (1, 'foo', 1, '2023-01-01'), (2, 'bar', 2, '2023-01-02'), (3, 'baz', 3, '2023-01-03')) @@ -1920,12 +2179,18 @@ physical_plan 09)│ format: csv │ 10)└─────────────┬─────────────┘ 11)┌─────────────┴─────────────┐ -12)│ DataSourceExec │ +12)│ yield_stream_exec │ 13)│ -------------------- │ -14)│ bytes: 2672 │ -15)│ format: memory │ -16)│ rows: 1 │ -17)└───────────────────────────┘ +14)│ YieldStreamExec child: │ +15)│ DataSourceExec │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ DataSourceExec │ +19)│ -------------------- │ +20)│ bytes: 2672 │ +21)│ format: memory │ +22)│ rows: 1 │ +23)└───────────────────────────┘ query TT explain COPY (VALUES (1, 'foo', 1, '2023-01-01'), (2, 'bar', 2, '2023-01-02'), (3, 'baz', 3, '2023-01-03')) @@ -1943,12 +2208,18 @@ physical_plan 09)│ format: arrow │ 10)└─────────────┬─────────────┘ 11)┌─────────────┴─────────────┐ -12)│ DataSourceExec │ +12)│ yield_stream_exec │ 13)│ -------------------- │ -14)│ bytes: 2672 │ -15)│ format: memory │ -16)│ rows: 1 │ -17)└───────────────────────────┘ +14)│ YieldStreamExec child: │ +15)│ DataSourceExec │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ DataSourceExec │ +19)│ -------------------- │ +20)│ bytes: 2672 │ +21)│ format: memory │ +22)│ rows: 1 │ +23)└───────────────────────────┘ # Test explain tree rendering for CoalesceBatchesExec with limit @@ -2012,12 +2283,18 @@ physical_plan 51)│ predicate: a > 3 │ 52)└─────────────┬─────────────┘ 53)┌─────────────┴─────────────┐ -54)│ DataSourceExec │ +54)│ yield_stream_exec │ 55)│ -------------------- │ -56)│ bytes: 160 │ -57)│ format: memory │ -58)│ rows: 1 │ -59)└───────────────────────────┘ +56)│ YieldStreamExec child: │ +57)│ DataSourceExec │ +58)└─────────────┬─────────────┘ +59)┌─────────────┴─────────────┐ +60)│ DataSourceExec │ +61)│ -------------------- │ +62)│ bytes: 160 │ +63)│ format: memory │ +64)│ rows: 1 │ +65)└───────────────────────────┘ # clean up statement ok @@ -2029,12 +2306,18 @@ EXPLAIN SELECT * FROM generate_series(1, 100) ---- physical_plan 01)┌───────────────────────────┐ -02)│ LazyMemoryExec │ +02)│ yield_stream_exec │ 03)│ -------------------- │ -04)│ batch_generators: │ -05)│ generate_series: start=1, │ -06)│ end=100, batch_size=8192 │ -07)└───────────────────────────┘ +04)│ YieldStreamExec child: │ +05)│ LazyMemoryExec │ +06)└─────────────┬─────────────┘ +07)┌─────────────┴─────────────┐ +08)│ LazyMemoryExec │ +09)│ -------------------- │ +10)│ batch_generators: │ +11)│ generate_series: start=1, │ +12)│ end=100, batch_size=8192 │ +13)└───────────────────────────┘ # Test explain tree for CoalescePartitionsExec query TT @@ -2069,11 +2352,17 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ +29)│ yield_stream_exec │ 30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +31)│ YieldStreamExec child: │ +32)│ StreamingTableExec │ +33)└─────────────┬─────────────┘ +34)┌─────────────┴─────────────┐ +35)│ StreamingTableExec │ +36)│ -------------------- │ +37)│ infinite: true │ +38)│ limit: None │ +39)└───────────────────────────┘ # Test explain tree for PlaceholderRowExec query TT @@ -2086,5 +2375,11 @@ physical_plan 04)│ count(*): 2 │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ PlaceholderRowExec │ -08)└───────────────────────────┘ +07)│ yield_stream_exec │ +08)│ -------------------- │ +09)│ YieldStreamExec child: │ +10)│ PlaceholderRowExec │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ PlaceholderRowExec │ +14)└───────────────────────────┘ diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index e4d0b7233856..73dd115eed0a 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -2088,7 +2088,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[min(t.a) FILTER (WHERE t.a > Int64(1))@0 as x] 02)--AggregateExec: mode=Single, gby=[], aggr=[min(t.a) FILTER (WHERE t.a > Int64(1))] -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index a09d8ce26ddf..749e38c1a44a 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -41,7 +41,8 @@ physical_plan 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] +05)--------YieldStreamExec child=StreamingTableExec +06)----------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 @@ -58,7 +59,8 @@ physical_plan 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] +05)--------YieldStreamExec child=StreamingTableExec +06)----------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 @@ -75,7 +77,8 @@ physical_plan 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] +05)--------YieldStreamExec child=StreamingTableExec +06)----------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 @@ -92,7 +95,8 @@ physical_plan 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] +05)--------YieldStreamExec child=StreamingTableExec +06)----------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 @@ -109,7 +113,8 @@ physical_plan 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] +05)--------YieldStreamExec child=StreamingTableExec +06)----------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 @@ -150,4 +155,5 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: date@0 = 2006-01-02 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] +05)--------YieldStreamExec child=StreamingTableExec +06)----------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 9e67018ecd0b..df68c1a16104 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2024,8 +2024,10 @@ physical_plan 09)----------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)] -12)----------------------DataSourceExec: partitions=1, partition_sizes=[3] -13)----------------------DataSourceExec: partitions=1, partition_sizes=[3] +12)----------------------YieldStreamExec child=DataSourceExec +13)------------------------DataSourceExec: partitions=1, partition_sizes=[3] +14)----------------------YieldStreamExec child=DataSourceExec +15)------------------------DataSourceExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is DataSourceExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2070,7 +2072,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@0 as a] 02)--SortExec: expr=[b@1 DESC], preserve_partitioning=[false] -03)----DataSourceExec: 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], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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], file_type=csv, has_header=true # Final plan shouldn't have SortExec c ASC, # because table already satisfies this ordering. @@ -2081,7 +2084,9 @@ logical_plan 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 DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # Final plan shouldn't have SortExec a ASC, b ASC, # because table already satisfies this ordering. @@ -2092,7 +2097,9 @@ logical_plan 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 DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # test_window_agg_sort statement ok @@ -2112,7 +2119,8 @@ logical_plan physical_plan 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] +03)----YieldStreamExec child=StreamingTableExec +04)------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 @@ -2143,7 +2151,8 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=PartiallySorted([1]) -03)----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] +03)----YieldStreamExec child=StreamingTableExec +04)------StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] query III SELECT a, d, @@ -2176,7 +2185,8 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], 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] +03)----YieldStreamExec child=StreamingTableExec +04)------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 @@ -2202,7 +2212,8 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], 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] +03)----YieldStreamExec child=StreamingTableExec +04)------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 @@ -2229,7 +2240,8 @@ logical_plan physical_plan 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] +03)----YieldStreamExec child=StreamingTableExec +04)------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 c) as last_c @@ -2251,7 +2263,8 @@ logical_plan 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan 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] +02)--YieldStreamExec child=StreamingTableExec +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 * @@ -2264,7 +2277,8 @@ logical_plan 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan 01)PartialSortExec: TopK(fetch=50), 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] +02)--YieldStreamExec child=StreamingTableExec +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 * @@ -2276,7 +2290,8 @@ logical_plan 02)--TableScan: multiple_ordered_table projection=[a0, a, b, c, d] physical_plan 01)SortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: 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]], file_type=csv, has_header=true +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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]], file_type=csv, has_header=true query TT EXPLAIN SELECT a, b, ARRAY_AGG(d ORDER BY d) @@ -2289,7 +2304,8 @@ logical_plan physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[array_agg(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]], 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] +03)----YieldStreamExec child=StreamingTableExec +04)------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? @@ -2461,7 +2477,8 @@ physical_plan 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) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query T? @@ -2491,7 +2508,8 @@ physical_plan 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) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)] 03)----SortExec: expr=[amount@1 DESC], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2536,7 +2554,8 @@ physical_plan 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) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted 03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST, amount@1 DESC], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort @@ -2574,7 +2593,8 @@ physical_plan 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) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=PartiallySorted([0]) 03)----SortExec: TopK(fetch=10), expr=[country@1 ASC NULLS LAST, amount@2 DESC], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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, @@ -2611,7 +2631,8 @@ physical_plan 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) ORDER BY [s.country DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted 03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2647,7 +2668,8 @@ physical_plan 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) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted 03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST, amount@1 DESC], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort @@ -2744,7 +2766,8 @@ physical_plan 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) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 03)----SortExec: expr=[amount@1 DESC], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query T?RR rowsort SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -2775,7 +2798,8 @@ physical_plan 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) ORDER BY [sales_global.amount ASC NULLS LAST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS amounts, @@ -2807,7 +2831,8 @@ physical_plan 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) ORDER BY [sales_global.amount ASC NULLS LAST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query TRR? SELECT country, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, @@ -2837,7 +2862,8 @@ physical_plan 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) ORDER BY [sales_global.ts DESC NULLS FIRST], array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@2 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query TR? SELECT country, SUM(amount ORDER BY ts DESC) AS sum1, @@ -2870,7 +2896,8 @@ logical_plan 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=[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(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2903,7 +2930,8 @@ logical_plan 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=[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(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2942,8 +2970,10 @@ physical_plan 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)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------------YieldStreamExec child=DataSourceExec +10)--------------DataSourceExec: 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 @@ -2988,7 +3018,8 @@ physical_plan 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) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3024,7 +3055,8 @@ physical_plan 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) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query TRR @@ -3060,7 +3092,8 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], 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]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3086,7 +3119,8 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], 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]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3111,7 +3145,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 @@ -3135,7 +3170,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------SortExec: expr=[ts@0 DESC], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 @@ -3159,7 +3195,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 05)--------SortExec: expr=[amount@0 ASC NULLS LAST], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3189,7 +3226,8 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 08)--------------SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[true] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] query T? SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3225,7 +3263,8 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 08)--------------SortExec: expr=[amount@1 DESC], preserve_partitioning=[true] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -3425,7 +3464,8 @@ physical_plan 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)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query IRI SELECT s.sn, s.amount, 2*s.sn @@ -3494,9 +3534,11 @@ physical_plan 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)--------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1, projection=[amount@1, sn@2, amount@3] -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -11)------------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +12)------------------YieldStreamExec child=DataSourceExec +13)--------------------DataSourceExec: partitions=1, partition_sizes=[1] query IRR SELECT r.sn, SUM(l.amount), r.amount @@ -3643,7 +3685,8 @@ physical_plan 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)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] query ITIPTRR @@ -3826,7 +3869,8 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[last_value(foo.x)] 04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query I SELECT FIRST_VALUE(x) @@ -3848,7 +3892,8 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[first_value(foo.x)] 04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: partitions=1, partition_sizes=[1] # Since both ordering requirements are satisfied, there shouldn't be # any SortExec in the final plan. @@ -3869,7 +3914,8 @@ physical_plan 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) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], first_value(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------DataSourceExec: 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]], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: 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]], file_type=csv, has_header=true query II rowsort SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, @@ -3903,7 +3949,9 @@ ORDER BY c ASC; logical_plan 01)Sort: multiple_ordered_table.c ASC NULLS LAST 02)--TableScan: multiple_ordered_table projection=[c] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -3937,10 +3985,12 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], 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)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, 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)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +07)--------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] +08)----------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] +09)------------YieldStreamExec child=DataSourceExec +10)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # reset partition number to 8. statement ok @@ -3981,7 +4031,8 @@ physical_plan 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)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # drop table multiple_ordered_table_with_pk statement ok @@ -4022,7 +4073,8 @@ physical_plan 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)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -4044,7 +4096,8 @@ physical_plan 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)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT c, sum1, SUM(b) OVER() as sumb @@ -4064,7 +4117,8 @@ physical_plan 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)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4095,10 +4149,12 @@ physical_plan 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)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, 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)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +08)------ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] +09)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +10)----------YieldStreamExec child=DataSourceExec +11)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4113,7 +4169,7 @@ EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 ---- logical_plan 01)Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 -02)--Cross Join: +02)--Cross Join: 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))]] @@ -4127,10 +4183,12 @@ physical_plan 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)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, 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)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +07)----ProjectionExec: expr=[c@0 as c, sum(multiple_ordered_table_with_pk.d)@1 as sum1] +08)------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +09)--------YieldStreamExec child=DataSourceExec +10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # we do not generate physical plan for Repartition yet (e.g Distribute By queries). query TT @@ -4169,10 +4227,12 @@ physical_plan 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)------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, 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)------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +06)--ProjectionExec: expr=[c@0 as c, a@1 as a, sum(multiple_ordered_table_with_pk.d)@2 as sum1] +07)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +08)------YieldStreamExec child=DataSourceExec +09)--------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # table scan should be simplified. query TT @@ -4187,7 +4247,8 @@ logical_plan physical_plan 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)----DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # limit should be simplified query TT @@ -4206,7 +4267,8 @@ physical_plan 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)------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 8; @@ -4244,7 +4306,8 @@ physical_plan 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)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: 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; @@ -4265,7 +4328,8 @@ physical_plan 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(x@0 AS Float64) as alias1], aggr=[] -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. statement ok @@ -4298,7 +4362,8 @@ physical_plan 05)--------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 06)----------AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] +08)--------------YieldStreamExec child=StreamingTableExec +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 @@ -4353,7 +4418,8 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[date_part(MONTH, ts@0) as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], file_type=csv, has_header=false +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], file_type=csv, has_header=false query I SELECT extract(month from ts) as months @@ -4393,7 +4459,8 @@ physical_plan 01)SortPreservingMergeExec: [name@0 DESC, time_chunks@1 DESC], fetch=5 02)--ProjectionExec: expr=[name@0 as name, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@1) as time_chunks] 03)----RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -04)------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] +04)------YieldStreamExec child=StreamingTableExec +05)--------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] statement ok drop table t1 @@ -4469,7 +4536,8 @@ physical_plan 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)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], file_type=csv, has_header=true +13)------------------------YieldStreamExec child=DataSourceExec +14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], file_type=csv, has_header=true # Use PostgreSQL dialect statement ok @@ -4642,7 +4710,8 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([c2@0], 8), input_partitions=8 06)----------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[max(timestamp_table.t1)], lim=[4] 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=4 -08)--------------DataSourceExec: 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], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: 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], file_type=csv, has_header=true # Clean up statement ok @@ -5031,7 +5100,8 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, b, c] physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[array_agg(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]], ordering_mode=Sorted -02)--DataSourceExec: 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]], file_type=csv, has_header=true +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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]], file_type=csv, has_header=true query II? SELECT a, b, ARRAY_AGG(c ORDER BY c DESC) @@ -5176,8 +5246,10 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }, ts@0, 946684800000000000) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))], aggr=[count(keywords_stream.keyword)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(keyword@0, keyword@1)] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------YieldStreamExec child=DataSourceExec +08)----------DataSourceExec: partitions=1, partition_sizes=[1] query PI SELECT @@ -5613,7 +5685,8 @@ logical_plan 02)--TableScan: t projection=[a, b, c, 😀] physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c, 😀@3 as 😀], aggr=[] -02)--DataSourceExec: partitions=1, partition_sizes=[0] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain select * from (select a, b from t) as c group by a, b; @@ -5624,7 +5697,8 @@ logical_plan 03)----TableScan: t projection=[a, b] physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[] -02)--DataSourceExec: partitions=1, partition_sizes=[0] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table t; diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 8a9c01d36308..de3116d5ded9 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -73,7 +73,8 @@ physical_plan 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)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I INSERT INTO table_without_values SELECT @@ -133,7 +134,8 @@ physical_plan 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)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true @@ -184,7 +186,8 @@ physical_plan 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)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I @@ -219,7 +222,8 @@ logical_plan physical_plan 01)DataSinkExec: sink=MemoryTable (partitions=1) 02)--SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query I 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 24982dfc28a7..8ac736576c49 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -130,7 +130,8 @@ physical_plan 01)DataSinkExec: sink=CsvSink(file_groups=[]) 02)--SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] 03)----ProjectionExec: expr=[column1@0 as a, column2@1 as b] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query I INSERT INTO ordered_insert_test values (5, 1), (4, 2), (7,7), (7,8), (7,9), (7,10), (3, 3), (2, 4), (1, 5); @@ -364,7 +365,8 @@ physical_plan 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)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I INSERT INTO table_without_values SELECT @@ -425,7 +427,8 @@ physical_plan 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)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true @@ -462,7 +465,8 @@ logical_plan physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) 02)--SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query I insert into table_without_values select c1 from aggregate_test_100 order by c1; diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 19763ab0083f..41ea1cdcd922 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -682,7 +682,9 @@ query TT explain select * from t1 join t2 on false; ---- logical_plan EmptyRelation -physical_plan EmptyExec +physical_plan +01)YieldStreamExec child=EmptyExec +02)--EmptyExec # Make batch size smaller than table row number. to introduce parallelism to the plan. statement ok @@ -698,9 +700,11 @@ logical_plan 03)--TableScan: t2 projection=[t2_id, t2_name, t2_int] physical_plan 01)CrossJoinExec -02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table IF EXISTS t1; @@ -778,8 +782,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[1] # Reset the configs to old values statement ok @@ -850,8 +856,10 @@ logical_plan physical_plan 01)ProjectionExec: expr=[emp_id@1 as emp_id, name@2 as name, dept_name@0 as dept_name] 02)--NestedLoopJoinExec: join_type=Right, filter=name@0 = Alice OR name@0 = Bob -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[1] query ITT SELECT e.emp_id, e.name, d.dept_name @@ -937,8 +945,10 @@ physical_plan 01)CrossJoinExec 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: name@1 = Alice OR name@1 = Bob -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)--DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--YieldStreamExec child=DataSourceExec +07)----DataSourceExec: partitions=1, partition_sizes=[1] # expect no row for Carol query ITT @@ -990,8 +1000,10 @@ physical_plan 05)--------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)----------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----------YieldStreamExec child=DataSourceExec +11)------------DataSourceExec: partitions=1, partition_sizes=[1] query ITT SELECT e.emp_id, e.name, d.dept_name @@ -1186,9 +1198,12 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0), (v1@1, v1@1)], projection=[v0@0, v1@1, v0@2, v2@4, v3@5, v4@6] -08)--------------DataSourceExec: partitions=1, partition_sizes=[0] -09)--------------DataSourceExec: partitions=1, partition_sizes=[0] -10)----DataSourceExec: partitions=1, partition_sizes=[0] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[0] +10)--------------YieldStreamExec child=DataSourceExec +11)----------------DataSourceExec: partitions=1, partition_sizes=[0] +12)----YieldStreamExec child=DataSourceExec +13)------DataSourceExec: partitions=1, partition_sizes=[0] @@ -1377,17 +1392,20 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(s.b AS Int64)@1, col1@1)], projection=[b@0, col0@2, col1@3, a@4] 04)------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] -07)--------CoalesceBatchesExec: target_batch_size=8192 -08)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] -09)------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] -10)--------------DataSourceExec: partitions=1, partition_sizes=[1] -11)------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] -12)--------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 -13)----------------CoalesceBatchesExec: target_batch_size=8192 -14)------------------FilterExec: y@1 = x@0 -15)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] +08)--------CoalesceBatchesExec: target_batch_size=8192 +09)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] +10)------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] +11)--------------YieldStreamExec child=DataSourceExec +12)----------------DataSourceExec: partitions=1, partition_sizes=[1] +13)------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] +14)--------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 +15)----------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------FilterExec: y@1 = x@0 +17)--------------------YieldStreamExec child=DataSourceExec +18)----------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table pairs; @@ -1440,8 +1458,10 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=4 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -12)------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +13)------YieldStreamExec child=DataSourceExec +14)--------DataSourceExec: partitions=1, partition_sizes=[1] query III SELECT * @@ -1463,8 +1483,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[1] query IIII SELECT * FROM t0, LATERAL (SELECT * FROM t1 WHERE t0.v0 = t1.v0); diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index a1efc1317b4a..7633ea77f9df 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -57,9 +57,11 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], file_type=csv, has_header=true -05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], file_type=csv, has_header=true +06)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)--------YieldStreamExec child=DataSourceExec +08)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # preserve_inner_join query IIII nosort @@ -98,11 +100,13 @@ physical_plan 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true -06)--------CoalesceBatchesExec: target_batch_size=8192 -07)----------FilterExec: d@3 = 3 -08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------------DataSourceExec: 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], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true +07)--------CoalesceBatchesExec: target_batch_size=8192 +08)----------FilterExec: d@3 = 3 +09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)--------------YieldStreamExec child=DataSourceExec +11)----------------DataSourceExec: 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], file_type=csv, 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 ccecb9494331..a3c4429ed93b 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1344,9 +1344,11 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)--------------YieldStreamExec child=DataSourceExec +11)----------------DataSourceExec: partitions=1, partition_sizes=[1] # Join on struct query TT @@ -1361,9 +1363,11 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s3@0, s4@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)------YieldStreamExec child=DataSourceExec +07)--------DataSourceExec: partitions=1, partition_sizes=[1] query ?? select join_t3.s3, join_t4.s4 @@ -1402,9 +1406,11 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------------YieldStreamExec child=DataSourceExec +12)------------------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1431,9 +1437,11 @@ physical_plan 08)--------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] 09)----------------CoalesceBatchesExec: target_batch_size=2 10)------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -12)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +13)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)----------------------YieldStreamExec child=DataSourceExec +15)------------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1497,10 +1505,12 @@ physical_plan 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)------------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------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)] +10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------YieldStreamExec child=DataSourceExec +12)------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1523,10 +1533,12 @@ physical_plan 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)------------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------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)] +10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------YieldStreamExec child=DataSourceExec +12)------------DataSourceExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1551,10 +1563,12 @@ physical_plan 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)------------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------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)] +10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------YieldStreamExec child=DataSourceExec +12)------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1577,10 +1591,12 @@ physical_plan 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)------------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------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)] +10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------YieldStreamExec child=DataSourceExec +12)------------DataSourceExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1603,10 +1619,12 @@ physical_plan 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)------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------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)] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1627,10 +1645,12 @@ physical_plan 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)------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------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)] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1656,9 +1676,11 @@ physical_plan 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)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)--------YieldStreamExec child=DataSourceExec +11)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1682,9 +1704,11 @@ physical_plan 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)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)--------YieldStreamExec child=DataSourceExec +11)----------DataSourceExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -1705,10 +1729,12 @@ logical_plan physical_plan 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)----DataSourceExec: 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)--------DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----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)] +06)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)--------YieldStreamExec child=DataSourceExec +08)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1727,10 +1753,12 @@ logical_plan physical_plan 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)----DataSourceExec: 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)--------DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----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)] +06)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)--------YieldStreamExec child=DataSourceExec +08)----------DataSourceExec: partitions=1, partition_sizes=[1] ##### # Config teardown @@ -2055,11 +2083,13 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)----CoalesceBatchesExec: target_batch_size=2 -09)------FilterExec: t1_id@0 > 10 -10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----CoalesceBatchesExec: target_batch_size=2 +10)------FilterExec: t1_id@0 > 10 +11)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------YieldStreamExec child=DataSourceExec +13)------------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2094,11 +2124,13 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------FilterExec: t1_id@0 > 22 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: 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)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--CoalesceBatchesExec: target_batch_size=2 +09)----FilterExec: t2_id@0 > 11 +10)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------YieldStreamExec child=DataSourceExec +12)----------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2569,9 +2601,11 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(millis@2, millis@2)] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)------YieldStreamExec child=DataSourceExec +07)--------DataSourceExec: partitions=1, partition_sizes=[1] # left_join_using_2 query II @@ -2744,12 +2778,14 @@ physical_plan 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)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)--SortExec: expr=[c1@0 ASC], preserve_partitioning=[true] -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)----------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--SortExec: expr=[c1@0 ASC], preserve_partitioning=[true] +09)----CoalesceBatchesExec: target_batch_size=2 +10)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------YieldStreamExec child=DataSourceExec +13)------------DataSourceExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_date32 inner sort merge join on data type (Date32) query DDRTDDRT rowsort @@ -2776,12 +2812,14 @@ physical_plan 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)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)----SortExec: expr=[c3@2 ASC], preserve_partitioning=[true] -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)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----SortExec: expr=[c3@2 ASC], preserve_partitioning=[true] +11)------CoalesceBatchesExec: target_batch_size=2 +12)--------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 +13)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)------------YieldStreamExec child=DataSourceExec +15)--------------DataSourceExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_decimal right join on data type (Decimal) query DDRTDDRT rowsort @@ -2835,10 +2873,12 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: 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 @@ -2871,10 +2911,12 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: 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 @@ -2928,10 +2970,12 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: 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 @@ -2964,10 +3008,12 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: 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 @@ -3022,10 +3068,12 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: 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 @@ -3039,10 +3087,12 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: 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 @@ -3094,10 +3144,12 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: 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 @@ -3111,10 +3163,12 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: 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 @@ -3196,11 +3250,13 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------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] 07)------------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] -08)--------------DataSourceExec: 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], file_type=csv, has_header=true -09)----CoalesceBatchesExec: target_batch_size=2 -10)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST -11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------DataSourceExec: 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], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: 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], file_type=csv, has_header=true +10)----CoalesceBatchesExec: target_batch_size=2 +11)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST +12)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)----------YieldStreamExec child=DataSourceExec +14)------------DataSourceExec: 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], file_type=csv, 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 @@ -3228,13 +3284,15 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------DataSourceExec: 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], file_type=csv, has_header=true -07)----CoalesceBatchesExec: target_batch_size=2 -08)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------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] -11)------------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] -12)--------------DataSourceExec: 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], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: 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], file_type=csv, has_header=true +08)----CoalesceBatchesExec: target_batch_size=2 +09)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST +10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------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] +12)------------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] +13)--------------YieldStreamExec child=DataSourceExec +14)----------------DataSourceExec: 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], file_type=csv, has_header=true statement ok set datafusion.optimizer.prefer_existing_sort = false; @@ -3273,14 +3331,16 @@ physical_plan 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)------------------DataSourceExec: 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], file_type=csv, has_header=true -11)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -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)------------------DataSourceExec: 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], file_type=csv, has_header=true +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: 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], file_type=csv, has_header=true +12)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] +13)--------CoalesceBatchesExec: target_batch_size=2 +14)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +15)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +16)--------------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] +17)----------------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] +18)------------------YieldStreamExec child=DataSourceExec +19)--------------------DataSourceExec: 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], file_type=csv, has_header=true statement ok set datafusion.optimizer.prefer_hash_join = true; @@ -3312,10 +3372,12 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] -03)----DataSourceExec: 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], file_type=csv, 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)--------DataSourceExec: 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], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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], file_type=csv, has_header=true +05)----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] +06)------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] +07)--------YieldStreamExec child=DataSourceExec +08)----------DataSourceExec: 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], file_type=csv, 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. @@ -3339,10 +3401,12 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], file_type=csv, 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)--------DataSourceExec: 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], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], file_type=csv, has_header=true +05)----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] +06)------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] +07)--------YieldStreamExec child=DataSourceExec +08)----------DataSourceExec: 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], file_type=csv, has_header=true query TT EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 @@ -3366,8 +3430,10 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] -05)--------DataSourceExec: 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], file_type=csv, has_header=true -06)--------DataSourceExec: 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], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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], file_type=csv, has_header=true +07)--------YieldStreamExec child=DataSourceExec +08)----------DataSourceExec: 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], file_type=csv, has_header=true # create a table where there more than one valid ordering # that describes table. @@ -3414,10 +3480,12 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], 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)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, 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)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +07)--------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] +08)----------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] +09)------------YieldStreamExec child=DataSourceExec +10)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # run query above in multiple partitions statement ok @@ -3457,11 +3525,13 @@ physical_plan 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)------------------------DataSourceExec: 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], file_type=csv, 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)------------------------DataSourceExec: 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], file_type=csv, has_header=true +13)------------------------YieldStreamExec child=DataSourceExec +14)--------------------------DataSourceExec: 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], file_type=csv, has_header=true +15)------------------CoalesceBatchesExec: target_batch_size=2 +16)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +17)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +18)------------------------YieldStreamExec child=DataSourceExec +19)--------------------------DataSourceExec: 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], file_type=csv, has_header=true query TT EXPLAIN SELECT * @@ -3476,9 +3546,11 @@ logical_plan 05)----TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=a@1 < a@0 -02)--DataSourceExec: 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], file_type=csv, has_header=true -03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)----DataSourceExec: 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], file_type=csv, has_header=true +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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], file_type=csv, has_header=true +04)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: 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], file_type=csv, has_header=true # Currently datafusion can pushdown filter conditions with scalar UDF into # cross join. @@ -3495,9 +3567,11 @@ logical_plan 05)----TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=example(CAST(a@0 AS Float64), CAST(a@1 AS Float64)) > 3 -02)--DataSourceExec: 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], file_type=csv, has_header=true -03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)----DataSourceExec: 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], file_type=csv, has_header=true +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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], file_type=csv, has_header=true +04)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: 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], file_type=csv, has_header=true #### # Config teardown @@ -3615,12 +3689,15 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] 04)------ProjectionExec: expr=[1 as e, 3 as f] -05)--------PlaceholderRowExec -06)------UnionExec -07)--------ProjectionExec: expr=[1 as c, 2 as d] -08)----------PlaceholderRowExec -09)--------ProjectionExec: expr=[1 as c, 3 as d] -10)----------PlaceholderRowExec +05)--------YieldStreamExec child=PlaceholderRowExec +06)----------PlaceholderRowExec +07)------UnionExec +08)--------ProjectionExec: expr=[1 as c, 2 as d] +09)----------YieldStreamExec child=PlaceholderRowExec +10)------------PlaceholderRowExec +11)--------ProjectionExec: expr=[1 as c, 3 as d] +12)----------YieldStreamExec child=PlaceholderRowExec +13)------------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3658,12 +3735,15 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] 04)------ProjectionExec: expr=[1 as e, 3 as f] -05)--------PlaceholderRowExec -06)------UnionExec -07)--------ProjectionExec: expr=[1 as c, 2 as d] -08)----------PlaceholderRowExec -09)--------ProjectionExec: expr=[1 as c, 3 as d] -10)----------PlaceholderRowExec +05)--------YieldStreamExec child=PlaceholderRowExec +06)----------PlaceholderRowExec +07)------UnionExec +08)--------ProjectionExec: expr=[1 as c, 2 as d] +09)----------YieldStreamExec child=PlaceholderRowExec +10)------------PlaceholderRowExec +11)--------ProjectionExec: expr=[1 as c, 3 as d] +12)----------YieldStreamExec child=PlaceholderRowExec +13)------------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3871,8 +3951,10 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] 04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------YieldStreamExec child=DataSourceExec +08)--------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3928,8 +4010,10 @@ physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------YieldStreamExec child=DataSourceExec +07)--------DataSourceExec: partitions=1, partition_sizes=[1] # Null build indices: @@ -3989,8 +4073,10 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] 04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------YieldStreamExec child=DataSourceExec +08)--------DataSourceExec: partitions=1, partition_sizes=[1] # Test CROSS JOIN LATERAL syntax (planning) @@ -4095,10 +4181,12 @@ physical_plan 03)----AggregateExec: mode=Single, gby=[sn@1 as sn, ts@0 as ts, amount@2 as amount, currency@3 as currency], aggr=[last_value(e.rate)] 04)------CoalesceBatchesExec: target_batch_size=3 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] -06)----------DataSourceExec: partitions=1, partition_sizes=[0] -07)----------CoalesceBatchesExec: target_batch_size=3 -08)------------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] -09)--------------DataSourceExec: partitions=1, partition_sizes=[0] +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1, partition_sizes=[0] +08)----------CoalesceBatchesExec: target_batch_size=3 +09)------------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] +10)--------------YieldStreamExec child=DataSourceExec +11)----------------DataSourceExec: partitions=1, partition_sizes=[0] statement ok DROP TABLE sales_global; @@ -4124,9 +4212,11 @@ logical_plan 04)----TableScan: right_table projection=[x, y, z] physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=a@0 < x@1 -02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)--SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[0] query TT EXPLAIN SELECT * FROM left_table JOIN right_table ON left_table.a= c2@1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[1] ## Test !join.on.is_empty() && join.filter.is_some() query TT @@ -4220,8 +4316,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)], filter=c2@0 >= c2@1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[1] ## Add more test cases for join limit pushdown statement ok @@ -4284,8 +4382,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, b@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], limit=2, file_type=csv, has_header=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], limit=2, file_type=csv, has_header=true +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true ###### ## RIGHT JOIN w/ LIMIT @@ -4318,8 +4418,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@0, b@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true ###### ## FULL JOIN w/ LIMIT @@ -4355,8 +4457,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(a@0, b@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true statement ok drop table t1; @@ -4397,8 +4501,10 @@ physical_plan 03)----ProjectionExec: expr=[] 04)------CoalesceBatchesExec: target_batch_size=3 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(binary_col@0, binary_col@0)] -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: partitions=1, partition_sizes=[1] # Test hash join sort push down # Issue: https://github.com/apache/datafusion/issues/13559 @@ -4430,10 +4536,12 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=3 06)----------FilterExec: b@1 > 3, projection=[a@0] 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] -10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)----------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] +11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------YieldStreamExec child=DataSourceExec +13)------------DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select * from test where a in (select a from test where b > 3) order by c desc nulls last; @@ -4454,10 +4562,12 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=3 06)----------FilterExec: b@1 > 3, projection=[a@0] 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] -10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)----------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] +11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------YieldStreamExec child=DataSourceExec +13)------------DataSourceExec: partitions=1, partition_sizes=[1] query III select * from test where a in (select a from test where b > 3) order by c desc nulls first; @@ -4497,8 +4607,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2, state@5] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT age FROM (SELECT * FROM person a join person b using (id, age, state)); @@ -4513,8 +4625,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[age@1] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT a.* FROM person a join person b using (id, age); @@ -4529,8 +4643,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT a.*, b.* FROM person a join person b using (id, age); @@ -4544,8 +4660,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT * FROM person a join person b using (id, age, state) join person c using (id, age, state); @@ -4566,9 +4684,12 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] 03)----CoalesceBatchesExec: target_batch_size=3 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] -05)--------DataSourceExec: partitions=1, partition_sizes=[0] -06)--------DataSourceExec: partitions=1, partition_sizes=[0] -07)----DataSourceExec: partitions=1, partition_sizes=[0] +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: partitions=1, partition_sizes=[0] +07)--------YieldStreamExec child=DataSourceExec +08)----------DataSourceExec: partitions=1, partition_sizes=[0] +09)----YieldStreamExec child=DataSourceExec +10)------DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT * FROM person a NATURAL JOIN lineitem b; @@ -4581,8 +4702,10 @@ logical_plan 05)----TableScan: lineitem projection=[c1] physical_plan 01)CrossJoinExec -02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[0] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)--YieldStreamExec child=DataSourceExec +05)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT * FROM lineitem JOIN lineitem as lineitem2 USING (c1) @@ -4596,8 +4719,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c1@0, c1@0)], projection=[c1@0] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[0] +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; @@ -4700,9 +4825,11 @@ logical_plan 05)------EmptyRelation physical_plan 01)CrossJoinExec -02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--ProjectionExec: expr=[1 as Int64(1)] -04)----PlaceholderRowExec +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)--ProjectionExec: expr=[1 as Int64(1)] +05)----YieldStreamExec child=PlaceholderRowExec +06)------PlaceholderRowExec statement count 0 drop table j1; @@ -4733,9 +4860,11 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(person.id + Int64(10)@1, orders.customer_id * Int64(2)@1)], projection=[id@0, customer_id@2] 03)----ProjectionExec: expr=[id@0 as id, CAST(id@0 AS Int64) + 10 as person.id + Int64(10)] -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)----ProjectionExec: expr=[customer_id@0 as customer_id, CAST(customer_id@0 AS Int64) * 2 as orders.customer_id * Int64(2)] -06)------DataSourceExec: partitions=1, partition_sizes=[0] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[0] +06)----ProjectionExec: expr=[customer_id@0 as customer_id, CAST(customer_id@0 AS Int64) * 2 as orders.customer_id * Int64(2)] +07)------YieldStreamExec child=DataSourceExec +08)--------DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index b46b8c49d662..1c642b654439 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -63,7 +63,8 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]}, file_type=json +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]}, file_type=json query ? SELECT mycol FROM single_nan @@ -145,4 +146,6 @@ query TT EXPLAIN SELECT id FROM json_partitioned_test WHERE part = 2 ---- logical_plan TableScan: json_partitioned_test projection=[id], full_filters=[json_partitioned_test.part = Int32(2)] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_json/part=2/data.json]]}, projection=[id], file_type=json +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_json/part=2/data.json]]}, projection=[id], file_type=json diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 2f8944f462a1..b898faf76c7c 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -313,7 +313,8 @@ logical_plan 04)------TableScan: t1 projection=[], fetch=14 physical_plan 01)ProjectionExec: expr=[0 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 11); @@ -332,7 +333,8 @@ logical_plan 04)------TableScan: t1 projection=[], fetch=11 physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -351,7 +353,8 @@ logical_plan 04)------TableScan: t1 projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec child=PlaceholderRowExec +03)----PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -379,7 +382,8 @@ physical_plan 07)------------GlobalLimitExec: skip=6, fetch=3 08)--------------CoalesceBatchesExec: target_batch_size=8192, fetch=9 09)----------------FilterExec: a@0 > 3 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] query I SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); @@ -409,7 +413,8 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] -06)----------DataSourceExec: partitions=1 +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1 statement ok set datafusion.explain.show_sizes = true; @@ -552,7 +557,9 @@ explain SELECT * FROM data LIMIT 3; logical_plan 01)Limit: skip=0, fetch=3 02)--TableScan: data projection=[column1, column2], fetch=3 -physical_plan StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=3, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] +physical_plan +01)YieldStreamExec child=StreamingTableExec +02)--StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=3, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] # Do not remove limit with Sort when skip is used @@ -565,7 +572,8 @@ logical_plan 03)----TableScan: data projection=[column1, column2] physical_plan 01)GlobalLimitExec: skip=3, fetch=3 -02)--StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=6, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] +02)--YieldStreamExec child=StreamingTableExec +03)----StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=6, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] statement ok @@ -641,7 +649,8 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(ordered_table.a)] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true # Applying offset & limit when multiple streams from union # the plan must still have a global limit to apply the offset @@ -667,11 +676,13 @@ physical_plan 04)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] 05)--------ProjectionExec: expr=[CAST(c@0 AS Int64) as c] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true -08)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] -09)--------ProjectionExec: expr=[CAST(d@0 AS Int64) as c] -10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[d], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +09)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] +10)--------ProjectionExec: expr=[CAST(d@0 AS Int64) as c] +11)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)------------YieldStreamExec child=DataSourceExec +13)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[d], file_type=csv, has_header=true # Applying LIMIT & OFFSET to subquery. query III @@ -744,8 +755,10 @@ physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] 02)--GlobalLimitExec: skip=0, fetch=10 03)----CrossJoinExec -04)------DataSourceExec: partitions=1, partition_sizes=[1], fetch=1 -05)------DataSourceExec: partitions=1, partition_sizes=[1], fetch=10 +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1], fetch=1 +06)------YieldStreamExec child=DataSourceExec +07)--------DataSourceExec: partitions=1, partition_sizes=[1], fetch=10 query IIII @@ -768,8 +781,10 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=0, fetch=2 02)--CrossJoinExec -03)----DataSourceExec: partitions=1, partition_sizes=[1], fetch=2 -04)----DataSourceExec: partitions=1, partition_sizes=[1], fetch=2 +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1], fetch=2 +05)----YieldStreamExec child=DataSourceExec +06)------DataSourceExec: partitions=1, partition_sizes=[1], fetch=2 statement ok drop table testSubQueryLimit; @@ -854,7 +869,8 @@ physical_plan 02)--SortExec: TopK(fetch=1000), expr=[part_key@1 ASC NULLS LAST], preserve_partitioning=[false] 03)----ProjectionExec: expr=[1 as foo, part_key@0 as part_key] 04)------CoalescePartitionsExec: fetch=1 -05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..794]]}, projection=[part_key], limit=1, file_type=parquet +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..794]]}, projection=[part_key], limit=1, file_type=parquet query I with selection as ( diff --git a/datafusion/sqllogictest/test_files/listing_table_statistics.slt b/datafusion/sqllogictest/test_files/listing_table_statistics.slt index 890d1f2e9250..f7add64610a4 100644 --- a/datafusion/sqllogictest/test_files/listing_table_statistics.slt +++ b/datafusion/sqllogictest/test_files/listing_table_statistics.slt @@ -35,7 +35,9 @@ query TT explain format indent select * from t; ---- logical_plan TableScan: t projection=[int_col, str_col] -physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/2.parquet]]}, projection=[int_col, str_col], file_type=parquet, statistics=[Rows=Exact(4), Bytes=Exact(288), [(Col[0]: Min=Exact(Int64(-1)) Max=Exact(Int64(3)) Null=Exact(0)),(Col[1]: Min=Exact(Utf8View("a")) Max=Exact(Utf8View("d")) Null=Exact(0))]] +physical_plan +01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:)]] +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/2.parquet]]}, projection=[int_col, str_col], file_type=parquet, statistics=[Rows=Exact(4), Bytes=Exact(288), [(Col[0]: Min=Exact(Int64(-1)) Max=Exact(Int64(3)) Null=Exact(0)),(Col[1]: Min=Exact(Utf8View("a")) Max=Exact(Utf8View("d")) Null=Exact(0))]] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 42a4ba621801..5fd3c3a6436d 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -115,7 +115,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int_field@0 > 0 -03)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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 e8700b1fea27..60e10e6e0012 100644 --- a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt +++ b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt @@ -47,7 +47,8 @@ physical_plan 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)------DataSourceExec: 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], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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], file_type=csv, has_header=true query TT EXPLAIN @@ -63,7 +64,8 @@ physical_plan 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)------DataSourceExec: 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], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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], file_type=csv, has_header=true # Cast to larger types as well as preserving ordering # doesn't invalidate lexicographical ordering. @@ -84,7 +86,8 @@ physical_plan 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)------DataSourceExec: 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], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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], file_type=csv, has_header=true # test for common rename query TT @@ -99,7 +102,8 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, a@0 as a_big, b@1 as b] -02)--DataSourceExec: 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], file_type=csv, has_header=true +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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], file_type=csv, has_header=true query TT EXPLAIN @@ -113,7 +117,8 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, a@0 as a_big, b@1 as b] -02)--DataSourceExec: 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], file_type=csv, has_header=true +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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], file_type=csv, has_header=true # test for cast Utf8 @@ -136,7 +141,8 @@ physical_plan 02)--SortExec: expr=[a_str@0 ASC NULLS LAST, b@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(a@0 AS Utf8) as a_str, b@1 as b] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: 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], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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], file_type=csv, has_header=true # We cannot determine a+b is ordered from the # invariant [a ASC, b ASC] is satisfied. Hence @@ -152,7 +158,8 @@ logical_plan 02)--TableScan: multiple_ordered_table projection=[a, b] physical_plan 01)SortExec: expr=[a@0 + b@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: 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], file_type=csv, has_header=true +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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], file_type=csv, 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. @@ -171,4 +178,5 @@ physical_plan 02)--SortExec: expr=[sum_expr@0 ASC NULLS LAST], preserve_partitioning=[true] 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)--------DataSourceExec: 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], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/operator.slt b/datafusion/sqllogictest/test_files/operator.slt index a651eda99684..051b81c3efe8 100644 --- a/datafusion/sqllogictest/test_files/operator.slt +++ b/datafusion/sqllogictest/test_files/operator.slt @@ -280,7 +280,8 @@ WHERE int64 < 5 AND uint64 < 5 AND float64 < 5 AND decimal < 5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 < 5 AND uint64@7 < 5 AND float64@9 < 5 AND decimal@10 < Some(500),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] ## < negative integer (expect no casts) query TT @@ -290,7 +291,8 @@ WHERE int64 < -5 AND uint64 < -5 AND float64 < -5 AND decimal < -5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 < -5 AND CAST(uint64@7 AS Decimal128(20, 0)) < Some(-5),20,0 AND float64@9 < -5 AND decimal@10 < Some(-500),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] ## < decimal (expect casts for integers to float) query TT @@ -300,7 +302,8 @@ WHERE int64 < 5.1 AND uint64 < 5.1 AND float64 < 5.1 AND decimal < 5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) < 5.1 AND CAST(uint64@7 AS Float64) < 5.1 AND float64@9 < 5.1 AND decimal@10 < Some(510),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] ## < negative decimal (expect casts for integers to float) query TT @@ -310,7 +313,8 @@ WHERE int64 < -5.1 AND uint64 < -5.1 AND float64 < -5.1 AND decimal < -5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) < -5.1 AND CAST(uint64@7 AS Float64) < -5.1 AND float64@9 < -5.1 AND decimal@10 < Some(-510),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] ############### Equality ############### @@ -323,7 +327,8 @@ WHERE int64 = 5 AND uint64 = 5 AND float64 = 5 AND decimal = 5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 = 5 AND uint64@7 = 5 AND float64@9 = 5 AND decimal@10 = Some(500),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] ## = negative integer (expect no casts) query TT @@ -333,7 +338,8 @@ WHERE int64 = -5 AND uint64 = -5 AND float64 = -5 AND decimal = -5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 = -5 AND CAST(uint64@7 AS Decimal128(20, 0)) = Some(-5),20,0 AND float64@9 = -5 AND decimal@10 = Some(-500),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] ## = decimal (expect casts for integers to float) query TT @@ -343,7 +349,8 @@ WHERE int64 = 5.1 AND uint64 = 5.1 AND float64 = 5.1 AND decimal = 5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) = 5.1 AND CAST(uint64@7 AS Float64) = 5.1 AND float64@9 = 5.1 AND decimal@10 = Some(510),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] ## = negative decimal (expect casts for integers to float) query TT @@ -353,7 +360,8 @@ WHERE int64 = -5.1 AND uint64 = -5.1 AND float64 = -5.1 AND decimal = -5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) = -5.1 AND CAST(uint64@7 AS Float64) = -5.1 AND float64@9 = -5.1 AND decimal@10 = Some(-510),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/options.slt b/datafusion/sqllogictest/test_files/options.slt index 71ff12e8cc50..04d52e7bc953 100644 --- a/datafusion/sqllogictest/test_files/options.slt +++ b/datafusion/sqllogictest/test_files/options.slt @@ -33,7 +33,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: c0@0 < 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] ## # test_disable_coalesce @@ -51,7 +52,8 @@ logical_plan 02)--TableScan: a projection=[c0] physical_plan 01)FilterExec: c0@0 < 1 -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.execution.coalesce_batches = true @@ -74,7 +76,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=1234 02)--FilterExec: c0@0 < 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 3fc90a6459f2..56a1bbd68f71 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -166,7 +166,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] 02)--SortExec: expr=[c2@1 ASC NULLS LAST, c3@2 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true query II SELECT c2, c3 FROM aggregate_test_100 ORDER BY c2, c3, c2 @@ -470,7 +471,8 @@ physical_plan 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)------DataSourceExec: 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]], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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]], file_type=csv, has_header=true statement ok drop table multiple_ordered_table; @@ -501,7 +503,8 @@ physical_plan 01)SortPreservingMergeExec: [db15@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@0, 1659537600000000000) as db15] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=false +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=false query TT EXPLAIN SELECT DATE_TRUNC('DAY', ts) as dt_day @@ -516,7 +519,8 @@ physical_plan 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)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=false +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=false statement ok drop table csv_with_timestamps; @@ -559,7 +563,8 @@ physical_plan 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)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT CEIL(c11) as ceil_c11 @@ -574,7 +579,8 @@ physical_plan 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)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT LOG(c12, c11) as log_c11_base_c12 @@ -589,7 +595,8 @@ physical_plan 01)SortPreservingMergeExec: [log_c11_base_c12@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[log(c12@1, CAST(c11@0 AS Float64)) as log_c11_base_c12] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------DataSourceExec: 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 NULLS LAST]], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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 NULLS LAST]], file_type=csv, has_header=true query TT EXPLAIN SELECT LOG(c11, c12) as log_c12_base_c11 @@ -604,7 +611,8 @@ physical_plan 01)SortPreservingMergeExec: [log_c12_base_c11@0 DESC NULLS LAST] 02)--ProjectionExec: expr=[log(CAST(c11@0 AS Float64), c12@1) as log_c12_base_c11] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------DataSourceExec: 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 NULLS LAST]], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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 NULLS LAST]], file_type=csv, has_header=true statement ok drop table aggregate_test_100; @@ -666,7 +674,9 @@ EXPLAIN SELECT o_orderkey, o_orderstatus FROM orders ORDER BY o_orderkey ASC logical_plan 01)Sort: orders.o_orderkey ASC NULLS LAST 02)--TableScan: orders projection=[o_orderkey, o_orderstatus] -physical_plan DataSourceExec: 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], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: 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], file_type=csv, has_header=true # Create external table with DDL ordered columns that are missing @@ -705,7 +715,8 @@ logical_plan 02)--TableScan: t1 projection=[id, name] physical_plan 01)SortExec: expr=[id@0 DESC, name@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] query IT SELECT * FROM t1 ORDER BY id ASC, id, name, id DESC; @@ -724,7 +735,8 @@ logical_plan 02)--TableScan: t1 projection=[id, name] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST, name@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] # Minimal reproduction of issue 5970 @@ -799,15 +811,17 @@ physical_plan 08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 09)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] 10)------------------ProjectionExec: expr=[column1@0 as t] -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -12)------ProjectionExec: expr=[1 as m, t@0 as t] -13)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] -14)----------CoalesceBatchesExec: target_batch_size=8192 -15)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 -16)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -17)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] -18)------------------ProjectionExec: expr=[column1@0 as t] -19)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +13)------ProjectionExec: expr=[1 as m, t@0 as t] +14)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] +15)----------CoalesceBatchesExec: target_batch_size=8192 +16)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 +17)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +18)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] +19)------------------ProjectionExec: expr=[column1@0 as t] +20)--------------------YieldStreamExec child=DataSourceExec +21)----------------------DataSourceExec: partitions=1, partition_sizes=[1] ##### # Multi column sorting with lists @@ -1047,7 +1061,8 @@ physical_plan 02)--SortExec: TopK(fetch=5), expr=[c_str@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(c@0 AS Utf8View) as c_str] 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true # Casting from numeric to numeric types preserves the ordering @@ -1077,7 +1092,8 @@ physical_plan 01)SortPreservingMergeExec: [c_bigint@0 ASC NULLS LAST], fetch=5 02)--ProjectionExec: expr=[CAST(c@0 AS Int64) as c_bigint] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table ordered_table; @@ -1113,7 +1129,8 @@ physical_plan 02)--SortExec: TopK(fetch=5), expr=[abs_c@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[abs(c@0) as abs_c] 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table ordered_table; @@ -1147,7 +1164,8 @@ physical_plan 01)SortPreservingMergeExec: [abs_c@0 ASC NULLS LAST], fetch=5 02)--ProjectionExec: expr=[abs(c@0) as abs_c] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true # Boolean to integer casts preserve the order. statement ok @@ -1173,7 +1191,8 @@ physical_plan 01)SortPreservingMergeExec: [c@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[CAST(inc_col@0 > desc_col@1 AS Int32) as c] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[inc_col, desc_col], output_orderings=[[inc_col@0 ASC NULLS LAST], [desc_col@1 DESC]], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[inc_col, desc_col], output_orderings=[[inc_col@0 ASC NULLS LAST], [desc_col@1 DESC]], file_type=csv, has_header=true # Union a query with the actual data and one with a constant query I @@ -1197,7 +1216,8 @@ physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] 02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true statement ok set datafusion.execution.use_row_number_estimates_to_optimize_partitioning = true; @@ -1216,7 +1236,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] 02)--SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true statement ok set datafusion.execution.use_row_number_estimates_to_optimize_partitioning = false; @@ -1236,7 +1257,8 @@ physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] 02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true # Test: inputs into union with different orderings @@ -1261,10 +1283,12 @@ physical_plan 02)--UnionExec 03)----SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a@2 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] 04)------ProjectionExec: expr=[b@1 as b, c@2 as c, a@0 as a, NULL as a0, d@3 as d] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[c@2 ASC NULLS LAST], file_type=csv, has_header=true -06)----SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a0@3 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] -07)------ProjectionExec: expr=[b@1 as b, c@2 as c, NULL as a, a0@0 as a0, d@3 as d] -08)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, b, c, d], output_ordering=[c@2 ASC NULLS LAST], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[c@2 ASC NULLS LAST], file_type=csv, has_header=true +07)----SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a0@3 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] +08)------ProjectionExec: expr=[b@1 as b, c@2 as c, NULL as a, a0@0 as a0, d@3 as d] +09)--------YieldStreamExec child=DataSourceExec +10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, b, c, d], output_ordering=[c@2 ASC NULLS LAST], file_type=csv, has_header=true # Test: run the query from above query IIIII @@ -1307,9 +1331,11 @@ physical_plan 02)--ProjectionExec: expr=[CASE WHEN name@0 = name1 THEN 0 WHEN name@0 = name2 THEN 0.5 END as a] 03)----UnionExec 04)------ProjectionExec: expr=[name1 as name] -05)--------PlaceholderRowExec -06)------ProjectionExec: expr=[name2 as name] -07)--------PlaceholderRowExec +05)--------YieldStreamExec child=PlaceholderRowExec +06)----------PlaceholderRowExec +07)------ProjectionExec: expr=[name2 as name] +08)--------YieldStreamExec child=PlaceholderRowExec +09)----------PlaceholderRowExec query R SELECT @@ -1347,7 +1373,9 @@ EXPLAIN SELECT c1, c2 FROM table_with_ordered_pk ORDER BY c1, c2; logical_plan 01)Sort: table_with_ordered_pk.c1 ASC NULLS LAST, table_with_ordered_pk.c2 ASC NULLS LAST 02)--TableScan: table_with_ordered_pk projection=[c1, c2] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], constraints=[PrimaryKey([0])], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], constraints=[PrimaryKey([0])], file_type=csv, has_header=true statement ok drop table table_with_ordered_pk; @@ -1376,7 +1404,8 @@ EXPLAIN SELECT c1, SUM(c2) as sum_c2 FROM table_with_ordered_not_null GROUP BY c physical_plan 01)ProjectionExec: expr=[c1@0 as c1, sum(table_with_ordered_not_null.c2)@1 as sum_c2] 02)--AggregateExec: mode=Single, gby=[c1@0 as c1], aggr=[sum(table_with_ordered_not_null.c2)], ordering_mode=Sorted -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table table_with_ordered_not_null; diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index abc6fdab3c8a..18c5a2b8fd46 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -91,7 +91,8 @@ logical_plan physical_plan 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], preserve_partitioning=[true] -03)----DataSourceExec: 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], file_type=parquet +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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], file_type=parquet # Tear down test_table: statement ok @@ -120,7 +121,8 @@ logical_plan 02)--TableScan: test_table projection=[int_col, string_col] physical_plan 01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] -02)--DataSourceExec: 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], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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], file_type=parquet # Add another file to the directory underlying test_table query I @@ -143,7 +145,8 @@ logical_plan physical_plan 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], preserve_partitioning=[true] -03)----DataSourceExec: 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], file_type=parquet +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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], file_type=parquet # Perform queries using MIN and MAX @@ -411,7 +414,8 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% statement ok @@ -459,7 +463,8 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -510,7 +515,8 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -625,7 +631,8 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 LIKE f% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] statement ok drop table foo diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 1b6ae13fbe77..b31e76db8816 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -86,7 +86,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] # When filter pushdown *is* enabled, ParquetExec can filter exactly, @@ -114,7 +115,8 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: b@1 > 2, projection=[a@0] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -06)----------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] # also test querying on columns that are not in all the files query T @@ -134,7 +136,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2 AND a_null_count@3 != row_count@2, required_guarantees=[] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2 AND a_null_count@3 != row_count@2, required_guarantees=[] query I @@ -153,7 +156,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] 02)--SortExec: expr=[b@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] ## cleanup statement ok @@ -213,14 +217,17 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: val@0 != part@1 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 -04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet # If we reference only a partition column it gets evaluted during the listing phase query TT EXPLAIN select * from t_pushdown where part != 'a'; ---- logical_plan TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part != Utf8("a")] -physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet # And if we reference only a file column it gets pushed down query TT @@ -229,7 +236,9 @@ EXPLAIN select * from t_pushdown where val != 'c'; logical_plan 01)Filter: t_pushdown.val != Utf8("c") 02)--TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != Utf8("c")] -physical_plan DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] # If we have a mix of filters: # - The partition filters get evaluated during planning @@ -245,4 +254,5 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: val@0 != part@1 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] diff --git a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt index a10243f62720..988798908dfa 100644 --- a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt @@ -111,7 +111,8 @@ logical_plan 02)--TableScan: test_table projection=[int_col, bigint_col] physical_plan 01)SortPreservingMergeExec: [int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Another planning test, but project on a column with unsupported statistics # We should be able to ignore this and look at only the relevant statistics @@ -128,7 +129,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[string_col@0 as string_col] 02)--SortPreservingMergeExec: [int_col@1 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[string_col, int_col, bigint_col], output_ordering=[int_col@1 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST], file_type=parquet +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[string_col, int_col, bigint_col], output_ordering=[int_col@1 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST], file_type=parquet # Clean up & recreate but sort on descending column statement ok @@ -162,7 +164,8 @@ logical_plan 02)--TableScan: test_table projection=[descending_col, bigint_col] physical_plan 01)SortPreservingMergeExec: [descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[descending_col, bigint_col], output_ordering=[descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[descending_col, bigint_col], output_ordering=[descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Clean up & re-create with partition columns in sort order statement ok @@ -198,7 +201,8 @@ logical_plan 02)--TableScan: test_table projection=[int_col, bigint_col, partition_col] physical_plan 01)SortPreservingMergeExec: [partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, partition_col], output_ordering=[partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, partition_col], output_ordering=[partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Clean up & re-create with overlapping column in sort order # This will test the ability to sort files with overlapping statistics @@ -231,7 +235,8 @@ logical_plan 02)--TableScan: test_table projection=[int_col, bigint_col, overlapping_col] physical_plan 01)SortPreservingMergeExec: [overlapping_col@2 ASC NULLS LAST] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, overlapping_col], output_ordering=[overlapping_col@2 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, overlapping_col], output_ordering=[overlapping_col@2 ASC NULLS LAST], file_type=parquet # Clean up & re-create with constant column in sort order # This will require a sort because the # of required file groups (3) @@ -266,4 +271,5 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [constant_col@0 ASC NULLS LAST] 02)--SortExec: expr=[constant_col@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[constant_col], file_type=parquet +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[constant_col], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/parquet_statistics.slt b/datafusion/sqllogictest/test_files/parquet_statistics.slt index c707b9f5bbd5..37ac5965d10f 100644 --- a/datafusion/sqllogictest/test_files/parquet_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_statistics.slt @@ -62,8 +62,9 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] 02)--FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] -05), statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +04)------YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] +06), statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] # cleanup statement ok @@ -86,11 +87,12 @@ query TT EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Inexact(2), Bytes=Inexact(44), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] -02)--FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(44), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(173), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] -05), statistics=[Rows=Inexact(5), Bytes=Inexact(173), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +02)--FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +04)------YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] +06), statistics=[Rows=Inexact(5), Bytes=Inexact(173), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] # cleanup statement ok @@ -117,8 +119,9 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] 02)--FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] -05), statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +04)------YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] +06), statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] # cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index b4b31fa78a69..0674366dcff0 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -675,13 +675,15 @@ physical_plan 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)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], file_type=csv, 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)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=true +10)----CoalesceBatchesExec: target_batch_size=8192 +11)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +12)--------CoalesceBatchesExec: target_batch_size=8192 +13)----------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 +14)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)--------------YieldStreamExec child=DataSourceExec +16)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], file_type=csv, has_header=true ######## # TPCH Q19 - Pull predicates to inner join (simplified) @@ -763,19 +765,22 @@ physical_plan 02)--ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey, ps_suppkey@0 as ps_suppkey] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, p_partkey@2)], projection=[ps_suppkey@1, l_extendedprice@2, l_discount@3, p_partkey@4] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)--------CoalesceBatchesExec: target_batch_size=8192 -07)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] -08)------------CoalesceBatchesExec: target_batch_size=8192 -09)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -10)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true -12)------------CoalesceBatchesExec: target_batch_size=8192 -13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)----------------CoalesceBatchesExec: target_batch_size=8192 -15)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------CoalesceBatchesExec: target_batch_size=8192 +08)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] +09)------------CoalesceBatchesExec: target_batch_size=8192 +10)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +11)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)------------------YieldStreamExec child=DataSourceExec +13)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true +14)------------CoalesceBatchesExec: target_batch_size=8192 +15)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +16)----------------CoalesceBatchesExec: target_batch_size=8192 +17)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +18)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)----------------------YieldStreamExec child=DataSourceExec +20)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true # Inlist simplification @@ -786,13 +791,17 @@ query TT explain select x from t where x IN (1,2,3) AND x IN (4,5); ---- logical_plan EmptyRelation -physical_plan EmptyExec +physical_plan +01)YieldStreamExec child=EmptyExec +02)--EmptyExec query TT explain select x from t where x NOT IN (1,2,3,4) OR x NOT IN (5,6,7,8); ---- logical_plan TableScan: t projection=[x] -physical_plan DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select x from t where x IN (1,2,3,4,5) AND x NOT IN (1,2,3,4); @@ -803,13 +812,16 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: x@0 = 5 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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); ---- logical_plan EmptyRelation -physical_plan EmptyExec +physical_plan +01)YieldStreamExec child=EmptyExec +02)--EmptyExec statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/projection.slt b/datafusion/sqllogictest/test_files/projection.slt index 0f0cbac1fa32..f9c52a7720ab 100644 --- a/datafusion/sqllogictest/test_files/projection.slt +++ b/datafusion/sqllogictest/test_files/projection.slt @@ -183,7 +183,9 @@ query TT explain select b from memory_table; ---- logical_plan TableScan: memory_table projection=[b] -physical_plan DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: partitions=1, partition_sizes=[1] query I select b from memory_table; @@ -248,7 +250,8 @@ logical_plan 02)--TableScan: t projection=[column1] physical_plan 01)ProjectionExec: expr=[get_field(column1@0, c0) as t.column1[c0]] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index ed948dd11439..244f19569970 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -42,7 +42,8 @@ physical_plan 04)------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------FilterExec: column1@0 = 2 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query I select uc2 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3; @@ -62,7 +63,8 @@ physical_plan 05)--------ProjectionExec: expr=[__unnest_placeholder(v.column2,depth=1)@0 as __unnest_placeholder(v.column2,depth=1)] 06)----------UnnestExec 07)------------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query II select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 AND column1 = 2; @@ -83,7 +85,8 @@ physical_plan 06)----------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------FilterExec: column1@0 = 2 -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query II select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 OR column1 = 2; @@ -103,7 +106,8 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------UnnestExec 06)----------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table v; @@ -127,7 +131,8 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------UnnestExec 06)----------ProjectionExec: expr=[column1@0 as column1, column2@1 as __unnest_placeholder(d.column2)] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -192,7 +197,8 @@ explain select * from test_filter_with_limit where value = 2 limit 1; ---- physical_plan 01)CoalescePartitionsExec: fetch=1 -02)--DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-2.parquet]]}, projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-2.parquet]]}, projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)] query II select * from test_filter_with_limit where value = 2 limit 1; @@ -229,43 +235,57 @@ LOCATION 'test_files/scratch/push_down_filter/t.parquet'; query TT explain select a from t where a = '100'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] # The predicate should not have a column cast when the value is a valid i32 query TT explain select a from t where a != '100'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 != 100, pruning_predicate=a_null_count@2 != row_count@3 AND (a_min@0 != 100 OR 100 != a_max@1), required_guarantees=[a not in (100)] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 != 100, pruning_predicate=a_null_count@2 != row_count@3 AND (a_min@0 != 100 OR 100 != a_max@1), required_guarantees=[a not in (100)] # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = '99999999999'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99999999999 +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99999999999 # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = '99.99'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99.99 +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99.99 # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = ''; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = # The predicate should not have a column cast when the operator is = or != and the literal can be round-trip casted without losing information. query TT explain select a from t where cast(a as string) = '100'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] # The predicate should still have the column cast when the literal alters its string representation after round-trip casting (leading zero lost). query TT explain select a from t where CAST(a AS string) = '0123'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 0123 +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 0123 statement ok diff --git a/datafusion/sqllogictest/test_files/references.slt b/datafusion/sqllogictest/test_files/references.slt index 0e72c5e5a29e..bbb3c8b52cdf 100644 --- a/datafusion/sqllogictest/test_files/references.slt +++ b/datafusion/sqllogictest/test_files/references.slt @@ -107,7 +107,8 @@ logical_plan physical_plan 01)SortExec: expr=[....@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[....@0 as ...., ....@0 as c3] -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] # Partial qualified name diff --git a/datafusion/sqllogictest/test_files/regexp/regexp_like.slt b/datafusion/sqllogictest/test_files/regexp/regexp_like.slt index 8c407ea2e760..d263bcd8ac9d 100644 --- a/datafusion/sqllogictest/test_files/regexp/regexp_like.slt +++ b/datafusion/sqllogictest/test_files/regexp/regexp_like.slt @@ -254,7 +254,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 LIKE %oo% -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] # Ensure casting / coercion works for all operators # (there should be no casts to Utf8) @@ -271,7 +272,8 @@ logical_plan 02)--TableScan: dict_table projection=[column1] physical_plan 01)ProjectionExec: expr=[column1@0 LIKE %oo% as dict_table.column1 LIKE Utf8("%oo%"), column1@0 NOT LIKE %oo% as dict_table.column1 NOT LIKE Utf8("%oo%"), column1@0 ILIKE %oo% as dict_table.column1 ILIKE Utf8("%oo%"), column1@0 NOT ILIKE %oo% as dict_table.column1 NOT ILIKE Utf8("%oo%")] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table strings diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 70666346e2ca..93fc64e5fa13 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -48,7 +48,8 @@ physical_plan 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)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # disable round robin repartitioning statement ok @@ -65,7 +66,8 @@ physical_plan 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)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # Cleanup @@ -125,7 +127,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 03)----FilterExec: c3@2 > 0 04)------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true +05)--------YieldStreamExec child=StreamingTableExec +06)----------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true # Start repratition on empty column test. # See https://github.com/apache/datafusion/issues/12057 diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index 2b30de572c8c..bf6f04c2f41f 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -61,7 +61,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok @@ -77,7 +78,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -102,7 +104,8 @@ physical_plan 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: column1@0 != 42 -05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:272..538, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..278], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:278..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:272..538, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..278], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:278..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered @@ -138,7 +141,8 @@ physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: column1@0 != 42 -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..269], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..273], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:273..547], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:269..538]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..269], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..273], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:273..547], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:269..538]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # Cleanup statement ok @@ -185,7 +189,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----DataSourceExec: 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], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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], file_type=csv, has_header=true # Cleanup statement ok @@ -228,7 +233,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----DataSourceExec: 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], file_type=json +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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], file_type=json # Cleanup statement ok @@ -253,7 +259,9 @@ query TT EXPLAIN SELECT * FROM arrow_table ---- logical_plan TableScan: arrow_table projection=[f0, f1, f2] -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:0..461], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:461..922], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:922..1383], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:1383..1842]]}, projection=[f0, f1, f2], file_type=arrow +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:0..461], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:461..922], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:922..1383], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:1383..1842]]}, projection=[f0, f1, f2], file_type=arrow # correct content query ITB @@ -285,7 +293,9 @@ query TT EXPLAIN SELECT * FROM avro_table ---- logical_plan TableScan: avro_table projection=[f1, f2, f3] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/simple_enum.avro]]}, projection=[f1, f2, f3], file_type=avro +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/simple_enum.avro]]}, projection=[f1, f2, f3], file_type=avro # Cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt index f583d659fd4f..90ba51d13642 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -1836,7 +1836,8 @@ logical_plan 02)--TableScan: simple_string projection=[letter] physical_plan 01)ProjectionExec: expr=[letter@0 as letter, letter@0 = A as simple_string.letter = left(Utf8("APACHE"),Int64(1))] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] query TB SELECT letter, letter = LEFT('APACHE', 1) FROM simple_string; @@ -1855,7 +1856,8 @@ logical_plan 02)--TableScan: simple_string projection=[letter, letter2] physical_plan 01)ProjectionExec: expr=[letter@0 as letter, letter@0 = left(letter2@1, 1) as simple_string.letter = left(simple_string.letter2,Int64(1))] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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 aa14faf984e4..8a11b6d1da43 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -424,19 +424,25 @@ query TT EXPLAIN VALUES (1, 'a', -1, 1.1),(NULL, 'b', -3, 0.5) ---- logical_plan Values: (Int64(1), Utf8("a"), Int64(-1), Float64(1.1)), (Int64(NULL), Utf8("b"), Int64(-3), Float64(0.5)) -physical_plan DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN VALUES ('1'::float) ---- logical_plan Values: (Float32(1) AS Utf8("1")) -physical_plan DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN VALUES (('1'||'2')::int unsigned) ---- logical_plan Values: (UInt32(12) AS Utf8("1") || Utf8("2")) -physical_plan DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: partitions=1, partition_sizes=[1] # all where empty @@ -565,8 +571,10 @@ logical_plan 06)------TableScan: foo projection=[column2] physical_plan 01)CrossJoinExec -02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)--YieldStreamExec child=DataSourceExec +05)----DataSourceExec: partitions=1, partition_sizes=[1] # select NaNs query BBBB @@ -933,7 +941,8 @@ logical_plan 02)--TableScan: select_between_data projection=[c1] physical_plan 01)ProjectionExec: expr=[c1@0 >= 2 AND c1@0 <= 3 as select_between_data.c1 BETWEEN Int64(2) AND Int64(3)] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] # TODO: query_get_indexed_field @@ -1196,7 +1205,9 @@ EXPLAIN SELECT a FROM annotated_data_finite2 logical_plan 01)Sort: annotated_data_finite2.a ASC NULLS LAST, fetch=5 02)--TableScan: annotated_data_finite2 projection=[a] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], limit=5, output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], limit=5, output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true query I SELECT a FROM annotated_data_finite2 @@ -1379,7 +1390,8 @@ logical_plan 02)--TableScan: table1 projection=[a] physical_plan 01)SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] # ambiguous column references in on join query error DataFusion error: Schema error: Ambiguous reference to unqualified field a @@ -1405,7 +1417,8 @@ physical_plan 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)------DataSourceExec: 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], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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], file_type=csv, has_header=true # since query below doesn't computation # inside projection expr, increasing partitions @@ -1422,7 +1435,8 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, b@1 as b, 2 as Int64(2)] -02)--DataSourceExec: 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], file_type=csv, has_header=true +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a @@ -1443,7 +1457,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: 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], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1464,7 +1479,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: 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], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1485,7 +1501,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: 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], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1506,7 +1523,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: 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], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is when filter contains or @@ -1528,7 +1546,8 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: a@1 = 0 OR b@2 = 0 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------DataSourceExec: 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], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: 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], file_type=csv, has_header=true # When ordering lost during projection, we shouldn't keep the SortExec. # in the final physical plan. @@ -1556,7 +1575,8 @@ physical_plan 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)------------ProjectionExec: expr=[c2@0 as c2] 08)--------------SortExec: TopK(fetch=4), expr=[c1@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c1], file_type=csv, has_header=true +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c1], file_type=csv, has_header=true # FilterExec can track equality of non-column expressions. # plan below shouldn't have a SortExec because given column 'a' is ordered. @@ -1576,7 +1596,8 @@ physical_plan 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)--------DataSourceExec: 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], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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], file_type=csv, has_header=true statement ok @@ -1604,7 +1625,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as t.x / Int64(2), __common_expr_1@0 + 1 as t.x / Int64(2) + Int64(1)] 02)--ProjectionExec: expr=[x@0 / 2 as __common_expr_1] -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT x/2, x/2+1 FROM t; @@ -1622,7 +1644,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as abs(t.x), __common_expr_1@0 + abs(y@1) as abs(t.x) + abs(t.y)] 02)--ProjectionExec: expr=[abs(x@0) as __common_expr_1, y@1 as y] -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT abs(x), abs(x) + abs(y) FROM t; @@ -1660,7 +1683,8 @@ logical_plan 02)--TableScan: t projection=[x, y] physical_plan 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)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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; @@ -1672,7 +1696,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 AND 1 / CAST(y@2 AS Int64) < 1 as t.y > Int64(0) AND Int64(1) / t.y < Int64(1), x@1 > 0 AND __common_expr_1@0 AND 1 / CAST(y@2 AS Int64) < 1 / CAST(x@1 AS Int64) as t.x > Int64(0) AND t.y > Int64(0) AND Int64(1) / t.y < Int64(1) / t.x] 02)--ProjectionExec: expr=[y@1 > 0 as __common_expr_1, x@0 as x, y@1 as y] -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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; @@ -1684,7 +1709,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 OR 1 / CAST(y@2 AS Int64) < 1 as t.y = Int64(0) OR Int64(1) / t.y < Int64(1), x@1 = 0 OR __common_expr_1@0 OR 1 / CAST(y@2 AS Int64) < 1 / CAST(x@1 AS Int64) as t.x = Int64(0) OR t.y = Int64(0) OR Int64(1) / t.y < Int64(1) / t.x] 02)--ProjectionExec: expr=[y@1 = 0 as __common_expr_1, x@0 as x, y@1 as y] -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] # due to the reason describe in https://github.com/apache/datafusion/issues/8927, # the following queries will fail diff --git a/datafusion/sqllogictest/test_files/simplify_expr.slt b/datafusion/sqllogictest/test_files/simplify_expr.slt index 075ccafcfd2e..056b4676d40e 100644 --- a/datafusion/sqllogictest/test_files/simplify_expr.slt +++ b/datafusion/sqllogictest/test_files/simplify_expr.slt @@ -28,7 +28,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: a@0 = 3 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] # test regex exprs query TT @@ -40,7 +41,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: b@0 IS NOT NULL -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select b from t where b !~ '.*' @@ -51,7 +53,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: b@0 = -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T select b from t where b ~ '.*' @@ -72,7 +75,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: a@0 IS NOT NULL OR NULL -03)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index c17fe8dfc7e6..950791bfe306 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -39,9 +39,11 @@ logical_plan physical_plan 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], preserve_partitioning=[false] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)--SortExec: expr=[a@0 ASC], preserve_partitioning=[false] -05)----DataSourceExec: partitions=1, partition_sizes=[1] +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)--SortExec: expr=[a@0 ASC], preserve_partitioning=[false] +06)----YieldStreamExec child=DataSourceExec +07)------DataSourceExec: 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 46e15a4d6d10..ef90c3ff8daa 100644 --- a/datafusion/sqllogictest/test_files/struct.slt +++ b/datafusion/sqllogictest/test_files/struct.slt @@ -124,7 +124,8 @@ logical_plan 02)--TableScan: values projection=[a, b, c] physical_plan 01)ProjectionExec: expr=[struct(a@0, b@1, c@2) as struct(values.a,values.b,values.c)] -02)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] # error on 0 arguments query error diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 796570633f67..044fd32fc1ee 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -210,9 +210,11 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------YieldStreamExec child=DataSourceExec +15)----------DataSourceExec: partitions=1, partition_sizes=[1] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -245,9 +247,11 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------YieldStreamExec child=DataSourceExec +15)----------DataSourceExec: partitions=1, partition_sizes=[1] 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 @@ -280,9 +284,11 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------YieldStreamExec child=DataSourceExec +15)----------DataSourceExec: partitions=1, partition_sizes=[1] 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 @@ -318,9 +324,11 @@ physical_plan 10)------------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------------------DataSourceExec: partitions=1, partition_sizes=[1] -14)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)--------DataSourceExec: partitions=1, partition_sizes=[1] +13)------------------------YieldStreamExec child=DataSourceExec +14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] +15)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)--------YieldStreamExec child=DataSourceExec +17)----------DataSourceExec: partitions=1, partition_sizes=[1] 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 @@ -773,9 +781,11 @@ physical_plan 01)ProjectionExec: expr=[count(*)@0 as b, count(Int64(1))@1 as count(Int64(1))] 02)--NestedLoopJoinExec: join_type=Left 03)----ProjectionExec: expr=[4 as count(*)] -04)------PlaceholderRowExec -05)----ProjectionExec: expr=[4 as count(Int64(1))] -06)------PlaceholderRowExec +04)------YieldStreamExec child=PlaceholderRowExec +05)--------PlaceholderRowExec +06)----ProjectionExec: expr=[4 as count(Int64(1))] +07)------YieldStreamExec child=PlaceholderRowExec +08)--------PlaceholderRowExec statement ok set datafusion.explain.logical_plan_only = true; @@ -1193,9 +1203,11 @@ physical_plan 02)--FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------YieldStreamExec child=DataSourceExec +09)------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/subquery_sort.slt b/datafusion/sqllogictest/test_files/subquery_sort.slt index d993515f4de9..4e8a1c9318c0 100644 --- a/datafusion/sqllogictest/test_files/subquery_sort.slt +++ b/datafusion/sqllogictest/test_files/subquery_sort.slt @@ -43,13 +43,17 @@ EXPLAIN SELECT c1 FROM (SELECT c1 FROM sink_table ORDER BY c2) AS ttt logical_plan 01)SubqueryAlias: ttt 02)--TableScan: sink_table projection=[c1] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query TT EXPLAIN SELECT c1 FROM (SELECT c1 FROM sink_table ORDER BY c2) ---- logical_plan TableScan: sink_table projection=[c1] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +physical_plan +01)YieldStreamExec child=DataSourceExec +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true # Do not remove ordering when it's with limit @@ -67,7 +71,8 @@ physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] 02)--SortExec: expr=[c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST, c9@3 ASC NULLS LAST], preserve_partitioning=[false] 03)----SortExec: TopK(fetch=2), expr=[c1@0 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true query TI @@ -102,7 +107,8 @@ physical_plan 03)----ProjectionExec: expr=[c1@0 as c1, rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as r, c3@1 as c3, c9@2 as c9] 04)------BoundedWindowAggExec: wdw=[rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "rank() ORDER BY [sink_table.c1 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(Utf8View(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 DESC], preserve_partitioning=[false] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3, c9], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3, c9], file_type=csv, has_header=true #Test with utf8view for window function statement ok @@ -128,7 +134,8 @@ physical_plan 03)----ProjectionExec: expr=[c1@0 as c1, rank() ORDER BY [sink_table_with_utf8view.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as r, c3@1 as c3, c9@2 as c9] 04)------BoundedWindowAggExec: wdw=[rank() ORDER BY [sink_table_with_utf8view.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "rank() ORDER BY [sink_table_with_utf8view.c1 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(Utf8View(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 DESC], preserve_partitioning=[false] -06)----------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok DROP TABLE sink_table_with_utf8view; @@ -154,7 +161,8 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true query TI diff --git a/datafusion/sqllogictest/test_files/table_functions.slt b/datafusion/sqllogictest/test_files/table_functions.slt index 7d318c50bacf..16dc8f2cea81 100644 --- a/datafusion/sqllogictest/test_files/table_functions.slt +++ b/datafusion/sqllogictest/test_files/table_functions.slt @@ -154,7 +154,9 @@ query TT EXPLAIN SELECT * FROM generate_series(1, 5) ---- logical_plan TableScan: tmp_table projection=[value] -physical_plan LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=5, batch_size=8192] +physical_plan +01)YieldStreamExec child=LazyMemoryExec +02)--LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=5, batch_size=8192] # # Test generate_series with invalid arguments @@ -271,7 +273,9 @@ query TT EXPLAIN SELECT * FROM range(1, 5) ---- logical_plan TableScan: tmp_table projection=[value] -physical_plan LazyMemoryExec: partitions=1, batch_generators=[range: start=1, end=5, batch_size=8192] +physical_plan +01)YieldStreamExec child=LazyMemoryExec +02)--LazyMemoryExec: partitions=1, batch_generators=[range: start=1, end=5, batch_size=8192] # # Test range with invalid arguments diff --git a/datafusion/sqllogictest/test_files/topk.slt b/datafusion/sqllogictest/test_files/topk.slt index ce23fe26528c..839293dc0b95 100644 --- a/datafusion/sqllogictest/test_files/topk.slt +++ b/datafusion/sqllogictest/test_files/topk.slt @@ -85,7 +85,8 @@ logical_plan 02)--TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] physical_plan 01)SortExec: TopK(fetch=5), expr=[c13@12 DESC], preserve_partitioning=[false] -02)--DataSourceExec: 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], file_type=csv, has_header=true +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: 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], file_type=csv, has_header=true @@ -316,7 +317,8 @@ explain select number, letter, age from partial_sorted order by number desc, let ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 ASC NULLS LAST, age@2 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC, letter@1 ASC NULLS LAST] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet # Explain variations of the above query with different orderings, and different sort prefixes. @@ -326,28 +328,32 @@ explain select number, letter, age from partial_sorted order by age desc limit 3 ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[age@2 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet query TT explain select number, letter, age from partial_sorted order by number desc, letter desc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet query TT explain select number, letter, age from partial_sorted order by number asc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet query TT explain select number, letter, age from partial_sorted order by letter asc, number desc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[letter@1 ASC NULLS LAST, number@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet # Explicit NULLS ordering cases (reversing the order of the NULLS on the number and letter orderings) query TT @@ -355,14 +361,16 @@ explain select number, letter, age from partial_sorted order by number desc, let ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 ASC], preserve_partitioning=[false], sort_prefix=[number@0 DESC] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet query TT explain select number, letter, age from partial_sorted order by number desc NULLS LAST, letter asc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC NULLS LAST, letter@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet # Verify that the sort prefix is correctly computed on the normalized ordering (removing redundant aliased columns) @@ -372,7 +380,8 @@ explain select number, letter, age, number as column4, letter as column5 from pa physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, column4@3 DESC, letter@1 ASC NULLS LAST, column5@4 ASC NULLS LAST, age@2 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC, letter@1 ASC NULLS LAST] 02)--ProjectionExec: expr=[number@0 as number, letter@1 as letter, age@2 as age, number@0 as column4, letter@1 as column5] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet # Verify that the sort prefix is correctly computed over normalized, order-maintaining projections (number + 1, number, number + 1, age) query TT @@ -384,7 +393,8 @@ physical_plan 03)----ProjectionExec: expr=[__common_expr_1@0 as number_plus, number@1 as number, __common_expr_1@0 as other_number_plus, age@2 as age] 04)------ProjectionExec: expr=[CAST(number@0 AS Int64) + 1 as __common_expr_1, number@0 as number, age@1 as age] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, age], output_ordering=[number@0 DESC], file_type=parquet +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, age], output_ordering=[number@0 DESC], file_type=parquet # Cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index d549f555f9d8..6df5b5d234fc 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -239,10 +239,13 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 05)--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 06)----------UnionExec -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------------ProjectionExec: expr=[name@0 || _new as name] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------------YieldStreamExec child=DataSourceExec 10)--------------DataSourceExec: partitions=1, partition_sizes=[1] +11)------------ProjectionExec: expr=[name@0 || _new as name] +12)--------------YieldStreamExec child=DataSourceExec +13)----------------DataSourceExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort @@ -270,10 +273,13 @@ logical_plan 05)----TableScan: t2 projection=[name] physical_plan 01)UnionExec -02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--DataSourceExec: partitions=1, partition_sizes=[1] -04)--ProjectionExec: expr=[name@0 || _new as name] +02)--YieldStreamExec child=DataSourceExec +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)--YieldStreamExec child=DataSourceExec 05)----DataSourceExec: partitions=1, partition_sizes=[1] +06)--ProjectionExec: expr=[name@0 || _new as name] +07)----YieldStreamExec child=DataSourceExec +08)------DataSourceExec: partitions=1, partition_sizes=[1] # Make sure to choose a small batch size to introduce parallelism to the plan. statement ok @@ -315,23 +321,27 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] -11)------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -12)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)----------DataSourceExec: partitions=1, partition_sizes=[1] -14)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] -15)----CoalesceBatchesExec: target_batch_size=2 -16)------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] -17)--------CoalescePartitionsExec -18)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -19)------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -20)--------------CoalesceBatchesExec: target_batch_size=2 -21)----------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -22)------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -23)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -24)----------------------DataSourceExec: partitions=1, partition_sizes=[1] -25)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -26)----------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +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)----------YieldStreamExec child=DataSourceExec +15)------------DataSourceExec: partitions=1, partition_sizes=[1] +16)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] +17)----CoalesceBatchesExec: target_batch_size=2 +18)------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] +19)--------CoalescePartitionsExec +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)----------------------YieldStreamExec child=DataSourceExec +27)------------------------DataSourceExec: partitions=1, partition_sizes=[1] +28)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)----------YieldStreamExec child=DataSourceExec +30)------------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort @@ -385,20 +395,24 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] -11)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)--------DataSourceExec: partitions=1, partition_sizes=[1] -13)--CoalesceBatchesExec: target_batch_size=2 -14)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)] -15)------CoalescePartitionsExec -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)------------------DataSourceExec: partitions=1, partition_sizes=[1] -22)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -23)--------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)--------YieldStreamExec child=DataSourceExec +14)----------DataSourceExec: partitions=1, partition_sizes=[1] +15)--CoalesceBatchesExec: target_batch_size=2 +16)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)] +17)------CoalescePartitionsExec +18)--------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +19)----------CoalesceBatchesExec: target_batch_size=2 +20)------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +21)--------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +22)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +23)------------------YieldStreamExec child=DataSourceExec +24)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +25)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +26)--------YieldStreamExec child=DataSourceExec +27)----------DataSourceExec: partitions=1, partition_sizes=[1] # union_upcast_types query TT @@ -417,11 +431,13 @@ physical_plan 03)----SortExec: TopK(fetch=5), expr=[c9@1 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[c1@0 as c1, CAST(c9@1 AS Decimal128(20, 0)) as c9] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true -07)----SortExec: TopK(fetch=5), expr=[c9@1 DESC], preserve_partitioning=[true] -08)------ProjectionExec: expr=[c1@0 as c1, CAST(c3@1 AS Decimal128(20, 0)) as c9] -09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true +08)----SortExec: TopK(fetch=5), expr=[c9@1 DESC], preserve_partitioning=[true] +09)------ProjectionExec: expr=[c1@0 as c1, CAST(c3@1 AS Decimal128(20, 0)) as c9] +10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)----------YieldStreamExec child=DataSourceExec +12)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true query TR SELECT c1, c9 FROM aggregate_test_100 UNION ALL SELECT c1, c3 FROM aggregate_test_100 ORDER BY c9 DESC LIMIT 5 @@ -458,13 +474,15 @@ physical_plan 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)----------------DataSourceExec: 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)----------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +12)--------CoalesceBatchesExec: target_batch_size=2 +13)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +14)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +15)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)----------------YieldStreamExec child=DataSourceExec +17)------------------DataSourceExec: partitions=1, partition_sizes=[1] # Union with limit push down 3 children test case query TT @@ -518,13 +536,16 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=2 13)------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true -16)----ProjectionExec: expr=[1 as cnt] -17)------PlaceholderRowExec -18)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] -19)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) 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 }], mode=[Sorted] -20)--------ProjectionExec: expr=[1 as c1] -21)----------PlaceholderRowExec +15)----------------------------YieldStreamExec child=DataSourceExec +16)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true +17)----ProjectionExec: expr=[1 as cnt] +18)------YieldStreamExec child=PlaceholderRowExec +19)--------PlaceholderRowExec +20)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] +21)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) 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 }], mode=[Sorted] +22)--------ProjectionExec: expr=[1 as c1] +23)----------YieldStreamExec child=PlaceholderRowExec +24)------------PlaceholderRowExec ######## @@ -603,9 +624,11 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST] 02)--UnionExec -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true -04)----ProjectionExec: expr=[c1a@0 as c1] -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1a], output_ordering=[c1a@0 ASC NULLS LAST], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +05)----ProjectionExec: expr=[c1a@0 as c1] +06)------YieldStreamExec child=DataSourceExec +07)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1a], output_ordering=[c1a@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table t1 @@ -630,11 +653,14 @@ physical_plan 01)UnionExec 02)--ProjectionExec: expr=[Int64(1)@0 as a] 03)----AggregateExec: mode=SinglePartitioned, gby=[1 as Int64(1)], aggr=[], ordering_mode=Sorted -04)------PlaceholderRowExec -05)--ProjectionExec: expr=[2 as a] -06)----PlaceholderRowExec -07)--ProjectionExec: expr=[3 as a] -08)----PlaceholderRowExec +04)------YieldStreamExec child=PlaceholderRowExec +05)--------PlaceholderRowExec +06)--ProjectionExec: expr=[2 as a] +07)----YieldStreamExec child=PlaceholderRowExec +08)------PlaceholderRowExec +09)--ProjectionExec: expr=[3 as a] +10)----YieldStreamExec child=PlaceholderRowExec +11)------PlaceholderRowExec # test UNION ALL aliases correctly with aliased subquery query TT @@ -659,10 +685,12 @@ physical_plan 02)--ProjectionExec: expr=[count(Int64(1))@1 as count, n@0 as n] 03)----AggregateExec: mode=SinglePartitioned, gby=[n@0 as n], aggr=[count(Int64(1))], ordering_mode=Sorted 04)------ProjectionExec: expr=[5 as n] -05)--------PlaceholderRowExec -06)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] -07)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] -08)------PlaceholderRowExec +05)--------YieldStreamExec child=PlaceholderRowExec +06)----------PlaceholderRowExec +07)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] +08)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] +09)------YieldStreamExec child=PlaceholderRowExec +10)--------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/11409 @@ -839,11 +867,13 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------FilterExec: c1@0 = a 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: 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], file_type=csv, has_header=true -07)----CoalesceBatchesExec: target_batch_size=2 -08)------FilterExec: c1@0 = a -09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)----------DataSourceExec: 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], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: 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], file_type=csv, has_header=true +08)----CoalesceBatchesExec: target_batch_size=2 +09)------FilterExec: c1@0 = a +10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)----------YieldStreamExec child=DataSourceExec +12)------------DataSourceExec: 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], file_type=csv, has_header=true # Clean up after the test statement ok @@ -916,9 +946,11 @@ physical_plan 03)----SortExec: expr=[y@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[CAST(y@0 AS Int64) as y] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----SortExec: expr=[y@0 ASC NULLS LAST], preserve_partitioning=[false] -08)------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)----SortExec: expr=[y@0 ASC NULLS LAST], preserve_partitioning=[false] +09)------YieldStreamExec child=DataSourceExec +10)--------DataSourceExec: partitions=1, partition_sizes=[1] # optimize_subquery_sort in create_relation removes Sort so the result is not sorted. query I @@ -943,8 +975,10 @@ physical_plan 01)UnionExec 02)--ProjectionExec: expr=[CAST(y@0 AS Int64) as y] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)--DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--YieldStreamExec child=DataSourceExec +07)----DataSourceExec: partitions=1, partition_sizes=[1] statement count 0 drop view v1; diff --git a/datafusion/sqllogictest/test_files/unnest.slt b/datafusion/sqllogictest/test_files/unnest.slt index 92e6f9995ae3..2e8a10864a55 100644 --- a/datafusion/sqllogictest/test_files/unnest.slt +++ b/datafusion/sqllogictest/test_files/unnest.slt @@ -605,7 +605,8 @@ physical_plan 03)----ProjectionExec: expr=[__unnest_placeholder(recursive_unnest_table.column3,depth=1)@0 as __unnest_placeholder(UNNEST(recursive_unnest_table.column3)), column3@1 as column3] 04)------UnnestExec 05)--------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] -06)----------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1, partition_sizes=[1] ## unnest->field_access->unnest->unnest query I? @@ -663,7 +664,8 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------UnnestExec 06)----------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index c86921012f9b..2c5af1259813 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -280,13 +280,17 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[max(d.a)], ordering_mode=Sorted 08)--------------UnionExec 09)----------------ProjectionExec: expr=[1 as a, aa as b] -10)------------------PlaceholderRowExec -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 +10)------------------YieldStreamExec child=PlaceholderRowExec +11)--------------------PlaceholderRowExec +12)----------------ProjectionExec: expr=[3 as a, aa as b] +13)------------------YieldStreamExec child=PlaceholderRowExec +14)--------------------PlaceholderRowExec +15)----------------ProjectionExec: expr=[5 as a, bb as b] +16)------------------YieldStreamExec child=PlaceholderRowExec +17)--------------------PlaceholderRowExec +18)----------------ProjectionExec: expr=[7 as a, bb as b] +19)------------------YieldStreamExec child=PlaceholderRowExec +20)--------------------PlaceholderRowExec # Check actual result: query TI @@ -366,13 +370,17 @@ physical_plan 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 +11)--------------------YieldStreamExec child=PlaceholderRowExec +12)----------------------PlaceholderRowExec +13)------------------ProjectionExec: expr=[3 as a, aa as b] +14)--------------------YieldStreamExec child=PlaceholderRowExec +15)----------------------PlaceholderRowExec +16)------------------ProjectionExec: expr=[5 as a, bb as b] +17)--------------------YieldStreamExec child=PlaceholderRowExec +18)----------------------PlaceholderRowExec +19)------------------ProjectionExec: expr=[7 as a, bb as b] +20)--------------------YieldStreamExec child=PlaceholderRowExec +21)----------------------PlaceholderRowExec # check actual result @@ -1245,7 +1253,8 @@ physical_plan 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], preserve_partitioning=[false] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], file_type=csv, has_header=true # over_order_by_sort_keys_sorting_prefix_compacting @@ -1266,7 +1275,8 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c2@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[false] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], file_type=csv, 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 @@ -1291,7 +1301,8 @@ physical_plan 05)--------SortExec: expr=[c9@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] 06)----------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 07)------------SortExec: expr=[c2@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[false] -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], file_type=csv, has_header=true # test_window_partition_by_order_by statement ok @@ -1321,7 +1332,8 @@ physical_plan 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)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], file_type=csv, has_header=true +12)----------------------YieldStreamExec child=DataSourceExec +13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], file_type=csv, has_header=true # test_window_agg_sort_reversed_plan @@ -1346,7 +1358,8 @@ physical_plan 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], preserve_partitioning=[false] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query III SELECT @@ -1387,7 +1400,8 @@ physical_plan 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], preserve_partitioning=[false] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query IIIIIII SELECT @@ -1431,7 +1445,8 @@ physical_plan 04)------SortExec: expr=[c9@0 ASC NULLS LAST], preserve_partitioning=[false] 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], preserve_partitioning=[false] -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query III @@ -1474,7 +1489,8 @@ physical_plan 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], preserve_partitioning=[false] -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], file_type=csv, has_header=true query IIII SELECT @@ -1565,7 +1581,8 @@ physical_plan 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], preserve_partitioning=[false] -17)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true +17)--------------------------------YieldStreamExec child=DataSourceExec +18)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true query IIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -1640,7 +1657,8 @@ physical_plan 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], preserve_partitioning=[false] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true query III @@ -1684,7 +1702,8 @@ physical_plan 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], preserve_partitioning=[false] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true query III SELECT @@ -1734,7 +1753,8 @@ physical_plan 07)------------SortExec: expr=[__common_expr_1@0 DESC, c9@3 DESC, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------ProjectionExec: expr=[c3@1 + c4@2 as __common_expr_1, c2@0 as c2, c3@1 as c3, c9@3 as c9] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], file_type=csv, has_header=true +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], file_type=csv, has_header=true query III @@ -1785,7 +1805,8 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=4096 11)--------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true +13)------------------------YieldStreamExec child=DataSourceExec +14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true query I @@ -1830,7 +1851,8 @@ physical_plan 08)--------------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] 09)----------------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] 10)------------------SortExec: expr=[c3@1 DESC, c9@2 DESC, c2@0 ASC NULLS LAST], preserve_partitioning=[false] -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], file_type=csv, has_header=true +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], file_type=csv, has_header=true @@ -1869,7 +1891,8 @@ physical_plan 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)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query TI SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC @@ -1998,7 +2021,8 @@ physical_plan 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)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true statement ok set datafusion.optimizer.repartition_sorts = true; @@ -2027,7 +2051,8 @@ physical_plan 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)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true # test_window_agg_with_global_limit statement ok @@ -2048,7 +2073,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(aggregate_test_100.c13)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST], preserve_partitioning=[false] -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], file_type=csv, has_header=true query ? @@ -2113,7 +2139,8 @@ physical_plan 06)----------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] 07)------------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 }] 08)--------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, c9@3 ASC NULLS LAST, c8@2 ASC NULLS LAST], preserve_partitioning=[false] -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], file_type=csv, has_header=true +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], file_type=csv, has_header=true @@ -2170,7 +2197,8 @@ physical_plan 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], preserve_partitioning=[false] 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)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], file_type=csv, has_header=true +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], file_type=csv, has_header=true query IIIII SELECT c9, @@ -2212,7 +2240,8 @@ physical_plan 05)--------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] 06)----------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] 07)------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[false] -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], file_type=csv, has_header=true query RR SELECT SUM(c12) OVER(ORDER BY c1, c2 GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as sum1, @@ -2246,7 +2275,8 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 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], preserve_partitioning=[false] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2283,7 +2313,8 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 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], preserve_partitioning=[false] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2320,7 +2351,8 @@ physical_plan 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], preserve_partitioning=[false] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2360,7 +2392,8 @@ physical_plan 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], preserve_partitioning=[false] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2435,7 +2468,8 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 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], preserve_partitioning=[false] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, 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. @@ -2457,7 +2491,8 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 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(Decimal128(None,21,0)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[CAST(c9@1 AS Decimal128(20, 0)) + CAST(c5@0 AS Decimal128(20, 0)) DESC], preserve_partitioning=[false] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], file_type=csv, has_header=true # Ordering equivalence should be preserved during cast expression query TT @@ -2478,7 +2513,8 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 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], preserve_partitioning=[false] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, 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 @@ -2586,7 +2622,8 @@ physical_plan 06)----------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, count(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: false, 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(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 07)------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, count(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, 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(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(8)), is_causal: false }], mode=[Sorted] 08)--------------ProjectionExec: expr=[CAST(desc_col@2 AS Int64) as __common_expr_1, CAST(inc_col@1 AS Int64) as __common_expr_2, ts@0 as ts, inc_col@1 as inc_col, desc_col@2 as desc_col] -09)----------------DataSourceExec: 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], file_type=csv, has_header=true +09)----------------YieldStreamExec child=DataSourceExec +10)------------------DataSourceExec: 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], file_type=csv, has_header=true query IIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -2669,7 +2706,8 @@ physical_plan 02)--ProjectionExec: expr=[ts@0 as ts, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] 03)----BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }], mode=[Sorted] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true query IIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -2742,7 +2780,8 @@ physical_plan 04)------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }], mode=[Sorted] 05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }], mode=[Sorted] 06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, CAST(inc_col@1 AS Float64) as __common_expr_2, ts@0 as ts, inc_col@1 as inc_col] -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true query IIIIIIIIRR SELECT @@ -2793,7 +2832,8 @@ physical_plan 03)----ProjectionExec: expr=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col] 04)------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 05)--------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true query IIIII SELECT @@ -2838,7 +2878,8 @@ physical_plan 03)----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: false, 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] 04)------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: false, 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] 05)--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] -06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +06)----------YieldStreamExec child=StreamingTableExec +07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII SELECT @@ -2883,7 +2924,8 @@ physical_plan 03)----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: false, 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] 04)------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: false, 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] 05)--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] -06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +06)----------YieldStreamExec child=StreamingTableExec +07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII @@ -2987,7 +3029,8 @@ physical_plan 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 __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -10)------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +10)------------------YieldStreamExec child=StreamingTableExec +11)--------------------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, @@ -3060,7 +3103,8 @@ physical_plan 12)----------------------SortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 13)------------------------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] 14)--------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -15)----------------------------DataSourceExec: 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], file_type=csv, has_header=true +15)----------------------------YieldStreamExec child=DataSourceExec +16)------------------------------DataSourceExec: 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], file_type=csv, has_header=true query IIIIIIIIIIIIIII SELECT a, b, c, @@ -3123,7 +3167,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=4096, fetch=5 03)----FilterExec: row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 < 50 04)------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] -05)--------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] +05)--------YieldStreamExec child=StreamingTableExec +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] # Top level sort is pushed down through BoundedWindowAggExec as its SUM result does already satisfy the required # global order. The existing sort is for the second-term lexicographical ordering requirement, which is being @@ -3146,7 +3191,8 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----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] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, 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 @@ -3235,7 +3281,8 @@ physical_plan 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 __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -08)--------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +08)--------------YieldStreamExec child=StreamingTableExec +09)----------------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; @@ -3275,7 +3322,8 @@ physical_plan 14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -17)--------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +17)--------------------------------YieldStreamExec child=StreamingTableExec +18)----------------------------------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 @@ -3334,7 +3382,8 @@ physical_plan 05)--------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] 06)----------WindowAggExec: wdw=[min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] 07)------------SortExec: expr=[c11@1 ASC NULLS LAST], preserve_partitioning=[false] -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], file_type=csv, has_header=true +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], file_type=csv, has_header=true # window1 spec is used multiple times under different aggregations. # The query should still work. @@ -3375,7 +3424,8 @@ physical_plan 03)----ProjectionExec: expr=[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min1, min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max1, c3@0 as c3] 04)------BoundedWindowAggExec: wdw=[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }, min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c12@1 ASC NULLS LAST], preserve_partitioning=[false] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], file_type=csv, has_header=true +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], file_type=csv, has_header=true # window2 spec is not defined statement error DataFusion error: Error during planning: The window window2 is not defined! @@ -3428,7 +3478,8 @@ physical_plan 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)------DataSourceExec: 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]], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: 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]], file_type=csv, has_header=true # Since column b is constant after filter b=0, # window requirement b ASC, d ASC can be satisfied @@ -3447,7 +3498,8 @@ physical_plan 02)--SortExec: expr=[d@4 ASC NULLS LAST], preserve_partitioning=[false] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------FilterExec: b@2 = 0 -05)--------DataSourceExec: 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]], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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]], file_type=csv, has_header=true # Create an unbounded source where there is multiple orderings. @@ -3483,7 +3535,8 @@ physical_plan 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)--------DataSourceExec: 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]], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: 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]], file_type=csv, has_header=true query TT EXPLAIN SELECT MAX(c) OVER(PARTITION BY d ORDER BY c ASC) as max_c @@ -3502,7 +3555,8 @@ physical_plan 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)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------YieldStreamExec child=DataSourceExec +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c ORDER BY a ASC) @@ -3515,7 +3569,8 @@ logical_plan physical_plan 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)----DataSourceExec: 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]], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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]], file_type=csv, has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) @@ -3528,7 +3583,8 @@ logical_plan physical_plan 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)----DataSourceExec: 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]], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: 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]], file_type=csv, has_header=true query I SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) @@ -3570,7 +3626,8 @@ physical_plan 01)ProjectionExec: expr=[c@0 as c, nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as nv1] 02)--GlobalLimitExec: skip=0, fetch=5 03)----WindowAggExec: wdw=[nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int32(NULL)), is_causal: false }] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true query II SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 @@ -3624,7 +3681,8 @@ physical_plan 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]] +07)------------YieldStreamExec child=StreamingTableExec +08)--------------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 @@ -3957,7 +4015,8 @@ physical_plan 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], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)------YieldStreamExec child=DataSourceExec +05)--------DataSourceExec: partitions=1, partition_sizes=[1] # test ROW_NUMBER window function returns correct data_type query T @@ -4078,7 +4137,8 @@ physical_plan 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], preserve_partitioning=[false] -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], file_type=csv, has_header=true query III SELECT c3, @@ -4121,7 +4181,8 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=4096 07)------------FilterExec: a@0 = 1 -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4144,7 +4205,8 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=4096 07)------------FilterExec: a@0 = 1 -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------YieldStreamExec child=DataSourceExec +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT @@ -5207,7 +5269,8 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------FilterExec: c1@0 = 2 OR c1@0 = 3 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5249,7 +5312,8 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5293,7 +5357,8 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=1 10)------------------FilterExec: c1@0 = 1 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)----------------------YieldStreamExec child=DataSourceExec +13)------------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5332,7 +5397,8 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------YieldStreamExec child=DataSourceExec +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5381,7 +5447,8 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=1 13)------------------------FilterExec: c1@0 > 1 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] +15)----------------------------YieldStreamExec child=DataSourceExec +16)------------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2 @@ -5432,7 +5499,8 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=1 13)------------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] +15)----------------------------YieldStreamExec child=DataSourceExec +16)------------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2 @@ -5492,7 +5560,8 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=1 05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT SUM(c9) OVER() as sum_c9 FROM aggregate_test_100_ordered ORDER BY sum_c9; @@ -5505,7 +5574,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as sum_c9] 02)--WindowAggExec: wdw=[sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100_ordered.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)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query TT EXPLAIN SELECT c1, MIN(c5) OVER(PARTITION BY c1) as min_c5 FROM aggregate_test_100_ordered ORDER BY c1, min_c5 DESC NULLS LAST; @@ -5522,7 +5592,8 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=1 05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT MAX(c5) OVER() as max_c5 FROM aggregate_test_100_ordered ORDER BY max_c5; @@ -5535,7 +5606,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as max_c5] 02)--WindowAggExec: wdw=[max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED 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(NULL)), is_causal: false }] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], file_type=csv, has_header=true +03)----YieldStreamExec child=DataSourceExec +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], file_type=csv, has_header=true query II rowsort SELECT From da58d0b355170a02e09f491f5255964de03f8a46 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 2 Jun 2025 21:36:33 +0800 Subject: [PATCH 19/74] Support final path --- .../src/wrap_leaves_cancellation.rs | 27 +- .../sqllogictest/test_files/aggregate.slt | 61 +- datafusion/sqllogictest/test_files/array.slt | 6 +- .../sqllogictest/test_files/arrow_files.slt | 8 +- datafusion/sqllogictest/test_files/binary.slt | 3 +- datafusion/sqllogictest/test_files/copy.slt | 9 +- .../test_files/count_star_rule.slt | 3 +- .../test_files/create_external_table.slt | 8 +- datafusion/sqllogictest/test_files/cse.slt | 33 +- datafusion/sqllogictest/test_files/cte.slt | 91 ++- datafusion/sqllogictest/test_files/ddl.slt | 8 +- .../sqllogictest/test_files/dictionary.slt | 9 +- .../sqllogictest/test_files/explain.slt | 159 ++--- .../sqllogictest/test_files/explain_tree.slt | 647 ++++++------------ .../test_files/filter_without_sort_exec.slt | 18 +- .../sqllogictest/test_files/group_by.slt | 84 +-- .../sqllogictest/test_files/join.slt.part | 65 +- .../join_disable_repartition_joins.slt | 20 +- datafusion/sqllogictest/test_files/joins.slt | 341 ++++----- datafusion/sqllogictest/test_files/json.slt | 4 +- datafusion/sqllogictest/test_files/limit.slt | 16 +- .../test_files/listing_table_statistics.slt | 4 +- datafusion/sqllogictest/test_files/map.slt | 3 +- .../test_files/monotonic_projection_test.slt | 15 +- .../sqllogictest/test_files/operator.slt | 24 +- .../sqllogictest/test_files/options.slt | 9 +- datafusion/sqllogictest/test_files/order.slt | 49 +- .../sqllogictest/test_files/parquet.slt | 15 +- .../test_files/parquet_filter_pushdown.slt | 14 +- .../test_files/parquet_sorted_statistics.slt | 15 +- .../test_files/parquet_statistics.slt | 21 +- .../sqllogictest/test_files/predicates.slt | 31 +- .../sqllogictest/test_files/projection.slt | 7 +- .../test_files/push_down_filter.slt | 46 +- .../test_files/regexp/regexp_like.slt | 6 +- .../sqllogictest/test_files/repartition.slt | 3 +- .../test_files/repartition_scan.slt | 23 +- datafusion/sqllogictest/test_files/scalar.slt | 6 +- datafusion/sqllogictest/test_files/select.slt | 55 +- .../sqllogictest/test_files/simplify_expr.slt | 12 +- datafusion/sqllogictest/test_files/struct.slt | 3 +- .../sqllogictest/test_files/subquery.slt | 28 +- .../sqllogictest/test_files/subquery_sort.slt | 8 +- .../test_files/table_functions.slt | 8 +- datafusion/sqllogictest/test_files/union.slt | 108 ++- datafusion/sqllogictest/test_files/unnest.slt | 6 +- datafusion/sqllogictest/test_files/window.slt | 42 +- 47 files changed, 761 insertions(+), 1420 deletions(-) diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index 1f6b6746158a..87a793e29787 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -18,6 +18,7 @@ use crate::PhysicalOptimizerRule; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; +use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::ExecutionPlan; use std::fmt::{Debug, Formatter}; @@ -44,22 +45,30 @@ impl WrapLeaves { fn wrap_recursive( &self, plan: Arc, + has_pipeline_breaking_above: bool, ) -> Result> { let children = plan.children(); + + let is_pipeline_breaker = plan.properties().emission_type == EmissionType::Final; + let should_wrap = has_pipeline_breaking_above; + if children.is_empty() { // Leaf node: wrap it in `YieldStreamExec` - let wrapped = Arc::new(YieldStreamExec::new(plan)); - Ok(wrapped) + if should_wrap { + Ok(Arc::new(YieldStreamExec::new(plan))) + } else { + Ok(plan) + } } else { - // Non-leaf: first process all children recursively let mut new_children = Vec::with_capacity(children.len()); for child in children { - let wrapped_child = self.wrap_recursive(Arc::clone(child))?; - new_children.push(wrapped_child); + let new_child = self.wrap_recursive( + Arc::clone(child), + has_pipeline_breaking_above || is_pipeline_breaker, + )?; + new_children.push(new_child); } - // Rebuild this node with the new children - let new_plan = plan.with_new_children(new_children)?; - Ok(new_plan) + Ok(plan.with_new_children(new_children)?) } } } @@ -87,7 +96,7 @@ impl PhysicalOptimizerRule for WrapLeaves { plan: Arc, _config: &ConfigOptions, ) -> Result> { - self.wrap_recursive(plan) + self.wrap_recursive(plan, false) } /// Since we only add `YieldStreamExec` wrappers (which preserve schema), schema_check remains true. diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 5d0258b130e1..ee8982907d93 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -394,20 +394,15 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[array_agg(DISTINCT a.foo), sum(DISTINCT Int64(1))], ordering_mode=Sorted 06)----------UnionExec 07)------------ProjectionExec: expr=[1 as id, 2 as foo] -08)--------------YieldStreamExec child=PlaceholderRowExec -09)----------------PlaceholderRowExec -10)------------ProjectionExec: expr=[1 as id, NULL as foo] -11)--------------YieldStreamExec child=PlaceholderRowExec -12)----------------PlaceholderRowExec -13)------------ProjectionExec: expr=[1 as id, NULL as foo] -14)--------------YieldStreamExec child=PlaceholderRowExec -15)----------------PlaceholderRowExec -16)------------ProjectionExec: expr=[1 as id, 3 as foo] -17)--------------YieldStreamExec child=PlaceholderRowExec -18)----------------PlaceholderRowExec -19)------------ProjectionExec: expr=[1 as id, 2 as foo] -20)--------------YieldStreamExec child=PlaceholderRowExec -21)----------------PlaceholderRowExec +08)--------------PlaceholderRowExec +09)------------ProjectionExec: expr=[1 as id, NULL as foo] +10)--------------PlaceholderRowExec +11)------------ProjectionExec: expr=[1 as id, NULL as foo] +12)--------------PlaceholderRowExec +13)------------ProjectionExec: expr=[1 as id, 3 as foo] +14)--------------PlaceholderRowExec +15)------------ProjectionExec: expr=[1 as id, 2 as foo] +16)--------------PlaceholderRowExec # FIX: custom absolute values @@ -6093,8 +6088,7 @@ logical_plan 02)--TableScan: empty projection=[col0] physical_plan 01)ProjectionExec: expr=[NULL as min(empty.col0)] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT EXPLAIN SELECT MAX(col0) FROM empty; @@ -6104,8 +6098,7 @@ logical_plan 02)--TableScan: empty projection=[col0] physical_plan 01)ProjectionExec: expr=[NULL as max(empty.col0)] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec statement ok DROP TABLE empty; @@ -6565,20 +6558,15 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[last_value(a.foo) ORDER BY [a.foo ASC NULLS LAST], sum(DISTINCT Int64(1))], ordering_mode=Sorted 06)----------UnionExec 07)------------ProjectionExec: expr=[1 as id, 2 as foo] -08)--------------YieldStreamExec child=PlaceholderRowExec -09)----------------PlaceholderRowExec -10)------------ProjectionExec: expr=[1 as id, 4 as foo] -11)--------------YieldStreamExec child=PlaceholderRowExec -12)----------------PlaceholderRowExec -13)------------ProjectionExec: expr=[1 as id, 5 as foo] -14)--------------YieldStreamExec child=PlaceholderRowExec -15)----------------PlaceholderRowExec -16)------------ProjectionExec: expr=[1 as id, 3 as foo] -17)--------------YieldStreamExec child=PlaceholderRowExec -18)----------------PlaceholderRowExec -19)------------ProjectionExec: expr=[1 as id, 2 as foo] -20)--------------YieldStreamExec child=PlaceholderRowExec -21)----------------PlaceholderRowExec +08)--------------PlaceholderRowExec +09)------------ProjectionExec: expr=[1 as id, 4 as foo] +10)--------------PlaceholderRowExec +11)------------ProjectionExec: expr=[1 as id, 5 as foo] +12)--------------PlaceholderRowExec +13)------------ProjectionExec: expr=[1 as id, 3 as foo] +14)--------------PlaceholderRowExec +15)------------ProjectionExec: expr=[1 as id, 2 as foo] +16)--------------PlaceholderRowExec # SortExec is removed if it is coming after one-row producing AggregateExec's having an empty group by expression query TT @@ -6711,8 +6699,7 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(Int64(1)), 2 as count()] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query II select count(1), count(*) from t; @@ -6728,8 +6715,7 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(Int64(1)), 2 as count(*)] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query II select count(), count(*) from t; @@ -6745,8 +6731,7 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(), 2 as count(*)] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT explain select count(1) * count(2) from t; diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 322b4a540696..756dbb9d53c1 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -7202,8 +7202,7 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[true as array_has_all(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(1),Int64(3)))] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec # array containment operator with scalars #2 (arrow at) query BBBBBBB @@ -7226,8 +7225,7 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[true as array_has_all(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(1),Int64(3)))] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec ### Array casting tests diff --git a/datafusion/sqllogictest/test_files/arrow_files.slt b/datafusion/sqllogictest/test_files/arrow_files.slt index 95ee4afe5528..30f322cf98fc 100644 --- a/datafusion/sqllogictest/test_files/arrow_files.slt +++ b/datafusion/sqllogictest/test_files/arrow_files.slt @@ -32,9 +32,7 @@ query TT EXPLAIN SELECT * FROM arrow_simple ---- logical_plan TableScan: arrow_simple projection=[f0, f1, f2] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[f0, f1, f2], file_type=arrow +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[f0, f1, f2], file_type=arrow # correct content query ITB @@ -119,9 +117,7 @@ query TT EXPLAIN SELECT f0 FROM arrow_partitioned WHERE part = 456 ---- logical_plan TableScan: arrow_partitioned projection=[f0], full_filters=[arrow_partitioned.part = Int32(456)] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_arrow/part=456/data.arrow]]}, projection=[f0], file_type=arrow +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_arrow/part=456/data.arrow]]}, projection=[f0], file_type=arrow # Errors in partition filters should be reported diff --git a/datafusion/sqllogictest/test_files/binary.slt b/datafusion/sqllogictest/test_files/binary.slt index 3631967396d6..1077c32e46f3 100644 --- a/datafusion/sqllogictest/test_files/binary.slt +++ b/datafusion/sqllogictest/test_files/binary.slt @@ -185,8 +185,7 @@ logical_plan 02)--TableScan: t projection=[column1] physical_plan 01)ProjectionExec: expr=[column1@0 as column1, column1@0 = 000102 as t.column1 = Binary("0,1,2")] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t_source diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index 90957b802a24..5eeb05e814ac 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -184,8 +184,7 @@ logical_plan 02)--TableScan: source_table projection=[col1, col2] physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: 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. @@ -199,8 +198,7 @@ logical_plan 02)--TableScan: source_table projection=[col1, col2] physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # Copy to directory as partitioned files with keep_partition_by_columns enabled query I @@ -658,8 +656,7 @@ logical_plan 03)----TableScan: t projection=[], fetch=10 physical_plan 01)DataSinkExec: sink=CsvSink(file_groups=[]) -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0], fetch=10 +02)--DataSourceExec: partitions=1, partition_sizes=[0], fetch=10 statement count 0 drop table t; diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index 15c5f6d2a2c6..8da0eb14c1c2 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -37,8 +37,7 @@ logical_plan 04)------EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as count()] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT EXPLAIN SELECT t1.a, COUNT() FROM t1 GROUP BY t1.a; diff --git a/datafusion/sqllogictest/test_files/create_external_table.slt b/datafusion/sqllogictest/test_files/create_external_table.slt index 7ce64bed2894..8075ff1b5884 100644 --- a/datafusion/sqllogictest/test_files/create_external_table.slt +++ b/datafusion/sqllogictest/test_files/create_external_table.slt @@ -253,9 +253,7 @@ EXPLAIN SELECT id FROM t ORDER BY id ASC; logical_plan 01)Sort: t.id ASC NULLS LAST 02)--TableScan: t projection=[id] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet ## Test a DESC order and verify that output_ordering is ASC from the previous OBRDER BY query TT @@ -281,9 +279,7 @@ query TT EXPLAIN SELECT id FROM t; ---- logical_plan TableScan: t projection=[id] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 DESC], file_type=parquet +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 DESC], file_type=parquet statement ok DROP TABLE t; diff --git a/datafusion/sqllogictest/test_files/cse.slt b/datafusion/sqllogictest/test_files/cse.slt index 02e04872f88b..1af4f14c937e 100644 --- a/datafusion/sqllogictest/test_files/cse.slt +++ b/datafusion/sqllogictest/test_files/cse.slt @@ -32,8 +32,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as c1, __common_expr_1@0 as c2] 02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Common volatile expression query TT @@ -47,8 +46,7 @@ logical_plan 02)--TableScan: t1 projection=[a] physical_plan 01)ProjectionExec: expr=[a@0 + random() as c1, a@0 + random() as c2] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] # Volatile expression with non-volatile common child query TT @@ -64,8 +62,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 + random() as c1, __common_expr_1@0 + random() as c2] 02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Volatile expression with non-volatile common children query TT @@ -81,8 +78,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 + random() + __common_expr_2@1 as c1, __common_expr_1@0 + random() + __common_expr_2@1 as c2] 02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1, a@0 + 2 as __common_expr_2] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Common short-circuit expression query TT @@ -104,8 +100,7 @@ physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as c1, __common_expr_1@0 as c2, __common_expr_2@1 as c3, __common_expr_2@1 as c4, __common_expr_3@2 as c5, __common_expr_3@2 as c6] 02)--ProjectionExec: expr=[__common_expr_4@0 AND b@1 = 0 as __common_expr_1, __common_expr_4@0 OR b@1 = 0 as __common_expr_2, CASE WHEN __common_expr_4@0 THEN 0 ELSE 1 END as __common_expr_3] 03)----ProjectionExec: expr=[a@0 = 0 as __common_expr_4, b@1 as b] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[0] +04)------DataSourceExec: partitions=1, partition_sizes=[0] # Common children of short-circuit expression query TT @@ -133,8 +128,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 AND b@4 = 0 as c1, __common_expr_1@0 AND b@4 = 1 as c2, b@4 = 2 AND a@3 = 1 as c3, b@4 = 3 AND a@3 = 1 as c4, __common_expr_2@1 OR b@4 = 4 as c5, __common_expr_2@1 OR b@4 = 5 as c6, b@4 = 6 OR a@3 = 3 as c7, b@4 = 7 OR a@3 = 3 as c8, CASE WHEN __common_expr_3@2 THEN 0 ELSE 1 END as c9, CASE WHEN __common_expr_3@2 THEN 0 ELSE 2 END as c10, CASE WHEN b@4 = 8 THEN a@3 + 1 ELSE 0 END as c11, CASE WHEN b@4 = 9 THEN a@3 + 1 ELSE 0 END as c12, CASE WHEN b@4 = 10 THEN 0 ELSE a@3 + 2 END as c13, CASE WHEN b@4 = 11 THEN 0 ELSE a@3 + 2 END as c14] 02)--ProjectionExec: expr=[a@0 = 0 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 = 4 as __common_expr_3, a@0 as a, b@1 as b] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Common children of volatile, short-circuit expression query TT @@ -162,8 +156,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 AND b@4 = random() as c1, __common_expr_1@0 AND b@4 = 1 + random() as c2, b@4 = 2 + random() AND a@3 = 1 as c3, b@4 = 3 + random() AND a@3 = 1 as c4, __common_expr_2@1 OR b@4 = 4 + random() as c5, __common_expr_2@1 OR b@4 = 5 + random() as c6, b@4 = 6 + random() OR a@3 = 3 as c7, b@4 = 7 + random() OR a@3 = 3 as c8, CASE WHEN __common_expr_3@2 THEN random() ELSE 1 END as c9, CASE WHEN __common_expr_3@2 THEN random() ELSE 2 END as c10, CASE WHEN b@4 = 8 + random() THEN a@3 + 1 ELSE 0 END as c11, CASE WHEN b@4 = 9 + random() THEN a@3 + 1 ELSE 0 END as c12, CASE WHEN b@4 = 10 + random() THEN 0 ELSE a@3 + 2 END as c13, CASE WHEN b@4 = 11 + random() THEN 0 ELSE a@3 + 2 END as c14] 02)--ProjectionExec: expr=[a@0 = 0 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 = 4 as __common_expr_3, a@0 as a, b@1 as b] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Common volatile children of short-circuit expression query TT @@ -181,8 +174,7 @@ logical_plan 02)--TableScan: t1 projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 = random() AND b@1 = 0 as c1, a@0 = random() AND b@1 = 1 as c2, a@0 = 2 + random() OR b@1 = 4 as c3, a@0 = 2 + random() OR b@1 = 5 as c4, CASE WHEN a@0 = 4 + random() THEN 0 ELSE 1 END as c5, CASE WHEN a@0 = 4 + random() THEN 0 ELSE 2 END as c6] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] # Surely only once but also conditionally evaluated expressions query TT @@ -202,8 +194,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[(__common_expr_1@0 OR random() = 0) AND __common_expr_2@1 as c1, __common_expr_2@1 AND random() = 0 OR __common_expr_1@0 as c2, CASE WHEN __common_expr_3@2 = 0 THEN __common_expr_3@2 ELSE 0 END as c3, CASE WHEN __common_expr_4@3 = 0 THEN 0 WHEN CAST(__common_expr_4@3 AS Boolean) THEN 0 ELSE 0 END as c4, CASE WHEN __common_expr_5@4 = 0 THEN 0 WHEN random() = 0 THEN __common_expr_5@4 ELSE 0 END as c5, CASE WHEN __common_expr_6@5 = 0 THEN 0 ELSE __common_expr_6@5 END as c6] 02)--ProjectionExec: expr=[a@0 = 1 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 + 3 as __common_expr_3, a@0 + 4 as __common_expr_4, a@0 + 5 as __common_expr_5, a@0 + 6 as __common_expr_6] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Surely only once but also conditionally evaluated subexpressions query TT @@ -221,8 +212,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[(__common_expr_1@0 OR random() = 0) AND (__common_expr_2@1 OR random() = 1) as c1, __common_expr_2@1 AND random() = 0 OR __common_expr_1@0 AND random() = 1 as c2, CASE WHEN __common_expr_3@2 = 0 THEN __common_expr_3@2 + random() ELSE 0 END as c3, CASE WHEN __common_expr_4@3 = 0 THEN 0 ELSE __common_expr_4@3 + random() END as c4] 02)--ProjectionExec: expr=[a@0 = 1 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 + 3 as __common_expr_3, a@0 + 4 as __common_expr_4] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Only conditionally evaluated expressions query TT @@ -240,5 +230,4 @@ logical_plan 02)--TableScan: t1 projection=[a] physical_plan 01)ProjectionExec: expr=[(random() = 0 OR a@0 = 1) AND a@0 = 2 as c1, random() = 0 AND a@0 = 2 OR a@0 = 1 as c2, CASE WHEN random() = 0 THEN a@0 + 3 ELSE a@0 + 3 END as c3, CASE WHEN random() = 0 THEN 0 WHEN a@0 + 4 = 0 THEN a@0 + 4 ELSE 0 END as c4, CASE WHEN random() = 0 THEN 0 WHEN a@0 + 5 = 0 THEN 0 ELSE a@0 + 5 END as c5, CASE WHEN random() = 0 THEN 0 WHEN random() = 0 THEN a@0 + 6 ELSE a@0 + 6 END as c6] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index a686e5c1cab5..6a07eb7d0a1a 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -36,8 +36,7 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec # cte_use_same_name_multiple_times statement error DataFusion error: Error during planning: WITH query name "a" specified more than once @@ -115,15 +114,13 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=nodes, is_distinct=false 02)--ProjectionExec: expr=[1 as id] -03)----YieldStreamExec child=PlaceholderRowExec -04)------PlaceholderRowExec -05)--CoalescePartitionsExec -06)----ProjectionExec: expr=[id@0 + 1 as id] -07)------CoalesceBatchesExec: target_batch_size=8192 -08)--------FilterExec: id@0 < 10 -09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------YieldStreamExec child=WorkTableExec -11)--------------WorkTableExec: name=nodes +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 @@ -736,19 +733,18 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=recursive_cte, is_distinct=false 02)--ProjectionExec: expr=[1 as val] -03)----YieldStreamExec child=PlaceholderRowExec -04)------PlaceholderRowExec -05)--ProjectionExec: expr=[2 as val] -06)----CrossJoinExec -07)------CoalescePartitionsExec -08)--------CoalesceBatchesExec: target_batch_size=8182 -09)----------FilterExec: val@0 < 2 -10)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------YieldStreamExec child=WorkTableExec -12)----------------WorkTableExec: name=recursive_cte -13)------ProjectionExec: expr=[2 as val] -14)--------YieldStreamExec child=PlaceholderRowExec -15)----------PlaceholderRowExec +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)--------------YieldStreamExec child=WorkTableExec +11)----------------WorkTableExec: name=recursive_cte +12)------ProjectionExec: expr=[2 as val] +13)--------YieldStreamExec child=PlaceholderRowExec +14)----------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/9794 # Non-recursive term and recursive term have different types @@ -880,8 +876,7 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT explain with numbers(a,b,c) as (select 1,2,3) select * from numbers; @@ -892,8 +887,7 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT explain with numbers as (select 1 as a, 2 as b, 3 as c) select * from numbers; @@ -904,8 +898,7 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec statement count 0 create table person (id int, name string, primary key(id)) @@ -922,10 +915,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8182 02)--HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(id@0, id@0)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; @@ -977,15 +968,13 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=numbers, is_distinct=false 02)--ProjectionExec: expr=[1 as n] -03)----YieldStreamExec child=PlaceholderRowExec -04)------PlaceholderRowExec -05)--CoalescePartitionsExec -06)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] -07)------CoalesceBatchesExec: target_batch_size=8182 -08)--------FilterExec: n@0 < 10 -09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------YieldStreamExec child=WorkTableExec -11)--------------WorkTableExec: name=numbers +03)----PlaceholderRowExec +04)--CoalescePartitionsExec +05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] +06)------CoalesceBatchesExec: target_batch_size=8182 +07)--------FilterExec: n@0 < 10 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------WorkTableExec: name=numbers query TT explain WITH RECURSIVE numbers AS ( @@ -1005,15 +994,13 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=numbers, is_distinct=false 02)--ProjectionExec: expr=[1 as n] -03)----YieldStreamExec child=PlaceholderRowExec -04)------PlaceholderRowExec -05)--CoalescePartitionsExec -06)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] -07)------CoalesceBatchesExec: target_batch_size=8182 -08)--------FilterExec: n@0 < 10 -09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------YieldStreamExec child=WorkTableExec -11)--------------WorkTableExec: name=numbers +03)----PlaceholderRowExec +04)--CoalescePartitionsExec +05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] +06)------CoalesceBatchesExec: target_batch_size=8182 +07)--------FilterExec: n@0 < 10 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------WorkTableExec: name=numbers statement count 0 set datafusion.execution.enable_recursive_ctes = false; diff --git a/datafusion/sqllogictest/test_files/ddl.slt b/datafusion/sqllogictest/test_files/ddl.slt index 674c463db4f1..1e95e426f3e0 100644 --- a/datafusion/sqllogictest/test_files/ddl.slt +++ b/datafusion/sqllogictest/test_files/ddl.slt @@ -713,9 +713,7 @@ query TT explain select c1 from t; ---- logical_plan TableScan: t projection=[c1] -physical_plan -01)YieldStreamExec child=StreamingTableExec -02)--StreamingTableExec: partition_sizes=1, projection=[c1], infinite_source=true +physical_plan StreamingTableExec: partition_sizes=1, projection=[c1], infinite_source=true statement ok drop table t; @@ -733,9 +731,7 @@ query TT explain select c1 from t; ---- logical_plan TableScan: t projection=[c1] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/empty.csv]]}, projection=[c1], file_type=csv, has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/empty.csv]]}, projection=[c1], file_type=csv, has_header=true statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt index 02be60626050..d241e61f33ff 100644 --- a/datafusion/sqllogictest/test_files/dictionary.slt +++ b/datafusion/sqllogictest/test_files/dictionary.slt @@ -412,8 +412,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column2@1 = 1 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # try literal = col to verify order doesn't matter # filter should not cast column2 @@ -426,8 +425,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column2@1 = 1 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # Now query using an integer which must be coerced into a dictionary string @@ -445,8 +443,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column2@1 = 1 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # Window Functions query I diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index d6e112422f42..010fdae34546 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -46,8 +46,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: c2@1 > 10, projection=[c1@0] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], file_type=csv, has_header=true # explain_csv_exec_scan_config @@ -78,9 +77,7 @@ explain SELECT c1 FROM aggregate_test_100_with_order order by c1 ASC limit 10 logical_plan 01)Sort: aggregate_test_100_with_order.c1 ASC NULLS LAST, fetch=10 02)--TableScan: aggregate_test_100_with_order projection=[c1] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_test_100_order_by_c1_asc.csv]]}, projection=[c1], limit=10, output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_test_100_order_by_c1_asc.csv]]}, projection=[c1], limit=10, output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true ## explain_physical_plan_only @@ -92,8 +89,7 @@ EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c ---- physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec statement ok set datafusion.explain.physical_plan_only = false @@ -134,9 +130,7 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test ---- logical_plan TableScan: simple_explain_test projection=[a, b, c] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true # create a sink table, path is same with aggregate_test_100 table # we do not overwrite this file, we only assert plan. @@ -249,19 +243,11 @@ physical_plan after OutputRequirements DataSourceExec: file_groups={1 group: [[W physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after wrap_leaves -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true +physical_plan after wrap_leaves SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true -physical_plan_with_stats -01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] -physical_plan_with_schema -01)YieldStreamExec child=DataSourceExec, schema=[a:Int32;N, b:Int32;N, c:Int32;N] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true +physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] ### tests for EXPLAIN with display schema enabled @@ -273,9 +259,7 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test; ---- logical_plan TableScan: simple_explain_test projection=[a, b, c] -physical_plan -01)YieldStreamExec child=DataSourceExec, schema=[a:Int32;N, b:Int32;N, c:Int32;N] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] statement ok @@ -293,9 +277,7 @@ set datafusion.explain.physical_plan_only = true; query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- -physical_plan -01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] # Parquet scan with statistics collected statement ok @@ -307,9 +289,7 @@ CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-t query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- -physical_plan -01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, 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)--DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(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 @@ -341,16 +321,10 @@ physical_plan after OutputRequirements physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(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 ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after wrap_leaves -01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, 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)--DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(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 wrap_leaves SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan -01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, 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)--DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan_with_schema -01)YieldStreamExec child=DataSourceExec, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] -02)--DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +physical_plan DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan_with_schema DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok @@ -389,19 +363,11 @@ physical_plan after OutputRequirements physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: 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, file_type=parquet physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after wrap_leaves -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: 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, file_type=parquet +physical_plan after wrap_leaves SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: 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, file_type=parquet -physical_plan_with_stats -01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, 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)--DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan_with_schema -01)YieldStreamExec child=DataSourceExec, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] -02)--DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +physical_plan DataSourceExec: 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, file_type=parquet +physical_plan_with_stats DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan_with_schema DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok @@ -420,8 +386,7 @@ logical_plan 02)--EmptyRelation physical_plan 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)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT explain select [[1, 2, 3], [4, 5, 6]]; @@ -431,8 +396,7 @@ logical_plan 02)--EmptyRelation physical_plan 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)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec # Explain Struct @@ -444,8 +408,7 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[{c0:1,c1:2.3,c2:abc} as struct(Int64(1),Float64(2.3),Utf8("abc"))] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec statement ok @@ -466,11 +429,9 @@ logical_plan 06)--------TableScan: t2 projection=[] physical_plan 01)NestedLoopJoinExec: join_type=LeftSemi -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)--ProjectionExec: expr=[] -05)----YieldStreamExec child=PlaceholderRowExec -06)------PlaceholderRowExec +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--ProjectionExec: expr=[] +04)----PlaceholderRowExec statement ok drop table t1; @@ -488,18 +449,14 @@ query TT explain select * from values (1); ---- logical_plan Values: (Int64(1)) -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] # can explicitly request indent mode query TT explain format indent select * from values (1); ---- logical_plan Values: (Int64(1)) -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] # tree mode query TT @@ -507,18 +464,12 @@ explain format tree select * from values (1); ---- physical_plan 01)┌───────────────────────────┐ -02)│ yield_stream_exec │ +02)│ DataSourceExec │ 03)│ -------------------- │ -04)│ YieldStreamExec child: │ -05)│ DataSourceExec │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ DataSourceExec │ -09)│ -------------------- │ -10)│ bytes: 128 │ -11)│ format: memory │ -12)│ rows: 1 │ -13)└───────────────────────────┘ +04)│ bytes: 128 │ +05)│ format: memory │ +06)│ rows: 1 │ +07)└───────────────────────────┘ # is not case sensitive query TT @@ -526,18 +477,12 @@ explain format TrEE select * from values (1); ---- physical_plan 01)┌───────────────────────────┐ -02)│ yield_stream_exec │ +02)│ DataSourceExec │ 03)│ -------------------- │ -04)│ YieldStreamExec child: │ -05)│ DataSourceExec │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ DataSourceExec │ -09)│ -------------------- │ -10)│ bytes: 128 │ -11)│ format: memory │ -12)│ rows: 1 │ -13)└───────────────────────────┘ +04)│ bytes: 128 │ +05)│ format: memory │ +06)│ rows: 1 │ +07)└───────────────────────────┘ # wrapped in single quotes query TT @@ -545,18 +490,12 @@ explain format 'tree' select * from values (1); ---- physical_plan 01)┌───────────────────────────┐ -02)│ yield_stream_exec │ +02)│ DataSourceExec │ 03)│ -------------------- │ -04)│ YieldStreamExec child: │ -05)│ DataSourceExec │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ DataSourceExec │ -09)│ -------------------- │ -10)│ bytes: 128 │ -11)│ format: memory │ -12)│ rows: 1 │ -13)└───────────────────────────┘ +04)│ bytes: 128 │ +05)│ format: memory │ +06)│ rows: 1 │ +07)└───────────────────────────┘ # wrapped in double quotes query TT @@ -564,18 +503,12 @@ explain format "tree" select * from values (1); ---- physical_plan 01)┌───────────────────────────┐ -02)│ yield_stream_exec │ +02)│ DataSourceExec │ 03)│ -------------------- │ -04)│ YieldStreamExec child: │ -05)│ DataSourceExec │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ DataSourceExec │ -09)│ -------------------- │ -10)│ bytes: 128 │ -11)│ format: memory │ -12)│ rows: 1 │ -13)└───────────────────────────┘ +04)│ bytes: 128 │ +05)│ format: memory │ +06)│ rows: 1 │ +07)└───────────────────────────┘ # number is not a valid format query error DataFusion error: SQL error: ParserError\("Expected: an explain format such as TREE, found: 123 at Line: 1, Column: 16"\) diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index bb30d538e2aa..c1ae8bc184c6 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -187,17 +187,11 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ yield_stream_exec │ +23)│ DataSourceExec │ 24)│ -------------------- │ -25)│ YieldStreamExec child: │ -26)│ DataSourceExec │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ DataSourceExec │ -30)│ -------------------- │ -31)│ files: 1 │ -32)│ format: csv │ -33)└───────────────────────────┘ +25)│ files: 1 │ +26)│ format: csv │ +27)└───────────────────────────┘ # Aggregate query TT @@ -274,17 +268,11 @@ physical_plan 05)│ skip: 3 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ yield_stream_exec │ +08)│ DataSourceExec │ 09)│ -------------------- │ -10)│ YieldStreamExec child: │ -11)│ DataSourceExec │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ DataSourceExec │ -15)│ -------------------- │ -16)│ files: 1 │ -17)│ format: csv │ -18)└───────────────────────────┘ +10)│ files: 1 │ +11)│ format: csv │ +12)└───────────────────────────┘ query TT explain SELECT * FROM limit_table LIMIT 10; @@ -296,18 +284,12 @@ physical_plan 04)│ limit: 10 │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ yield_stream_exec │ +07)│ DataSourceExec │ 08)│ -------------------- │ -09)│ YieldStreamExec child: │ -10)│ DataSourceExec │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ DataSourceExec │ -14)│ -------------------- │ -15)│ bytes: 3120 │ -16)│ format: memory │ -17)│ rows: 2 │ -18)└───────────────────────────┘ +09)│ bytes: 3120 │ +10)│ format: memory │ +11)│ rows: 2 │ +12)└───────────────────────────┘ # 2 Joins query TT @@ -351,17 +333,11 @@ physical_plan 35)│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ 36)└─────────────┬─────────────┘└─────────────┬─────────────┘ 37)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -38)│ yield_stream_exec ││ yield_stream_exec │ +38)│ DataSourceExec ││ DataSourceExec │ 39)│ -------------------- ││ -------------------- │ -40)│ YieldStreamExec child: ││ YieldStreamExec child: │ -41)│ DataSourceExec ││ DataSourceExec │ -42)└─────────────┬─────────────┘└─────────────┬─────────────┘ -43)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -44)│ DataSourceExec ││ DataSourceExec │ -45)│ -------------------- ││ -------------------- │ -46)│ files: 1 ││ files: 1 │ -47)│ format: csv ││ format: parquet │ -48)└───────────────────────────┘└───────────────────────────┘ +40)│ files: 1 ││ files: 1 │ +41)│ format: csv ││ format: parquet │ +42)└───────────────────────────┘└───────────────────────────┘ # 3 Joins query TT @@ -395,18 +371,18 @@ physical_plan 19)│ (int_col = int_col) │ │ 20)└─────────────┬─────────────┘ │ 21)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -22)│ yield_stream_exec ││ CoalesceBatchesExec │ +22)│ DataSourceExec ││ CoalesceBatchesExec │ 23)│ -------------------- ││ -------------------- │ -24)│ YieldStreamExec child: ││ target_batch_size: │ -25)│ DataSourceExec ││ 8192 │ -26)└─────────────┬─────────────┘└─────────────┬─────────────┘ -27)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -28)│ DataSourceExec ││ HashJoinExec │ -29)│ -------------------- ││ -------------------- │ -30)│ bytes: 1560 ││ on: ├──────────────┐ -31)│ format: memory ││ (int_col = int_col) │ │ -32)│ rows: 1 ││ │ │ -33)└───────────────────────────┘└─────────────┬─────────────┘ │ +24)│ bytes: 1560 ││ target_batch_size: │ +25)│ format: memory ││ 8192 │ +26)│ rows: 1 ││ │ +27)└───────────────────────────┘└─────────────┬─────────────┘ +28)-----------------------------┌─────────────┴─────────────┐ +29)-----------------------------│ HashJoinExec │ +30)-----------------------------│ -------------------- │ +31)-----------------------------│ on: ├──────────────┐ +32)-----------------------------│ (int_col = int_col) │ │ +33)-----------------------------└─────────────┬─────────────┘ │ 34)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 35)-----------------------------│ CoalesceBatchesExec ││ CoalesceBatchesExec │ 36)-----------------------------│ -------------------- ││ -------------------- │ @@ -432,17 +408,11 @@ physical_plan 56)-----------------------------│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ 57)-----------------------------└─────────────┬─────────────┘└─────────────┬─────────────┘ 58)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -59)-----------------------------│ yield_stream_exec ││ yield_stream_exec │ +59)-----------------------------│ DataSourceExec ││ DataSourceExec │ 60)-----------------------------│ -------------------- ││ -------------------- │ -61)-----------------------------│ YieldStreamExec child: ││ YieldStreamExec child: │ -62)-----------------------------│ DataSourceExec ││ DataSourceExec │ -63)-----------------------------└─────────────┬─────────────┘└─────────────┬─────────────┘ -64)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -65)-----------------------------│ DataSourceExec ││ DataSourceExec │ -66)-----------------------------│ -------------------- ││ -------------------- │ -67)-----------------------------│ files: 1 ││ files: 1 │ -68)-----------------------------│ format: csv ││ format: parquet │ -69)-----------------------------└───────────────────────────┘└───────────────────────────┘ +61)-----------------------------│ files: 1 ││ files: 1 │ +62)-----------------------------│ format: csv ││ format: parquet │ +63)-----------------------------└───────────────────────────┘└───────────────────────────┘ # Long Filter (demonstrate what happens with wrapping) query TT @@ -477,17 +447,11 @@ physical_plan 24)│ RoundRobinBatch(4) │ 25)└─────────────┬─────────────┘ 26)┌─────────────┴─────────────┐ -27)│ yield_stream_exec │ +27)│ DataSourceExec │ 28)│ -------------------- │ -29)│ YieldStreamExec child: │ -30)│ DataSourceExec │ -31)└─────────────┬─────────────┘ -32)┌─────────────┴─────────────┐ -33)│ DataSourceExec │ -34)│ -------------------- │ -35)│ files: 1 │ -36)│ format: csv │ -37)└───────────────────────────┘ +29)│ files: 1 │ +30)│ format: csv │ +31)└───────────────────────────┘ # Check maximum line limit. query TT @@ -545,17 +509,11 @@ physical_plan 48)│ RoundRobinBatch(4) │ 49)└─────────────┬─────────────┘ 50)┌─────────────┴─────────────┐ -51)│ yield_stream_exec │ +51)│ DataSourceExec │ 52)│ -------------------- │ -53)│ YieldStreamExec child: │ -54)│ DataSourceExec │ -55)└─────────────┬─────────────┘ -56)┌─────────────┴─────────────┐ -57)│ DataSourceExec │ -58)│ -------------------- │ -59)│ files: 1 │ -60)│ format: csv │ -61)└───────────────────────────┘ +53)│ files: 1 │ +54)│ format: csv │ +55)└───────────────────────────┘ # Check exactly the render width. query TT @@ -585,17 +543,11 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ yield_stream_exec │ +23)│ DataSourceExec │ 24)│ -------------------- │ -25)│ YieldStreamExec child: │ -26)│ DataSourceExec │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ DataSourceExec │ -30)│ -------------------- │ -31)│ files: 1 │ -32)│ format: csv │ -33)└───────────────────────────┘ +25)│ files: 1 │ +26)│ format: csv │ +27)└───────────────────────────┘ # Check with the render witdth + 1. query TT @@ -627,17 +579,11 @@ physical_plan 22)│ RoundRobinBatch(4) │ 23)└─────────────┬─────────────┘ 24)┌─────────────┴─────────────┐ -25)│ yield_stream_exec │ +25)│ DataSourceExec │ 26)│ -------------------- │ -27)│ YieldStreamExec child: │ -28)│ DataSourceExec │ -29)└─────────────┬─────────────┘ -30)┌─────────────┴─────────────┐ -31)│ DataSourceExec │ -32)│ -------------------- │ -33)│ files: 1 │ -34)│ format: csv │ -35)└───────────────────────────┘ +27)│ files: 1 │ +28)│ format: csv │ +29)└───────────────────────────┘ # Query with filter on csv query TT @@ -666,17 +612,11 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ yield_stream_exec │ +23)│ DataSourceExec │ 24)│ -------------------- │ -25)│ YieldStreamExec child: │ -26)│ DataSourceExec │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ DataSourceExec │ -30)│ -------------------- │ -31)│ files: 1 │ -32)│ format: csv │ -33)└───────────────────────────┘ +25)│ files: 1 │ +26)│ format: csv │ +27)└───────────────────────────┘ # Query with filter on parquet @@ -706,20 +646,14 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ yield_stream_exec │ +23)│ DataSourceExec │ 24)│ -------------------- │ -25)│ YieldStreamExec child: │ -26)│ DataSourceExec │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ DataSourceExec │ -30)│ -------------------- │ -31)│ files: 1 │ -32)│ format: parquet │ -33)│ │ -34)│ predicate: │ -35)│ string_col != foo │ -36)└───────────────────────────┘ +25)│ files: 1 │ +26)│ format: parquet │ +27)│ │ +28)│ predicate: │ +29)│ string_col != foo │ +30)└───────────────────────────┘ # Query with filter on memory query TT @@ -739,18 +673,12 @@ physical_plan 11)│ string_col != foo │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ yield_stream_exec │ +14)│ DataSourceExec │ 15)│ -------------------- │ -16)│ YieldStreamExec child: │ -17)│ DataSourceExec │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ DataSourceExec │ -21)│ -------------------- │ -22)│ bytes: 1560 │ -23)│ format: memory │ -24)│ rows: 1 │ -25)└───────────────────────────┘ +16)│ bytes: 1560 │ +17)│ format: memory │ +18)│ rows: 1 │ +19)└───────────────────────────┘ # Query with filter on json query TT @@ -779,17 +707,11 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ yield_stream_exec │ +23)│ DataSourceExec │ 24)│ -------------------- │ -25)│ YieldStreamExec child: │ -26)│ DataSourceExec │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ DataSourceExec │ -30)│ -------------------- │ -31)│ files: 1 │ -32)│ format: json │ -33)└───────────────────────────┘ +25)│ files: 1 │ +26)│ format: json │ +27)└───────────────────────────┘ # Query with filter on arrow query TT @@ -818,17 +740,11 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ yield_stream_exec │ +23)│ DataSourceExec │ 24)│ -------------------- │ -25)│ YieldStreamExec child: │ -26)│ DataSourceExec │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ DataSourceExec │ -30)│ -------------------- │ -31)│ files: 1 │ -32)│ format: arrow │ -33)└───────────────────────────┘ +25)│ files: 1 │ +26)│ format: arrow │ +27)└───────────────────────────┘ # Query with window agg. @@ -1049,17 +965,11 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ yield_stream_exec │ +23)│ DataSourceExec │ 24)│ -------------------- │ -25)│ YieldStreamExec child: │ -26)│ DataSourceExec │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ DataSourceExec │ -30)│ -------------------- │ -31)│ files: 1 │ -32)│ format: csv │ -33)└───────────────────────────┘ +25)│ files: 1 │ +26)│ format: csv │ +27)└───────────────────────────┘ query TT explain select @@ -1170,17 +1080,11 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ yield_stream_exec │ +23)│ DataSourceExec │ 24)│ -------------------- │ -25)│ YieldStreamExec child: │ -26)│ DataSourceExec │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ DataSourceExec │ -30)│ -------------------- │ -31)│ files: 1 │ -32)│ format: parquet │ -33)└───────────────────────────┘ +25)│ files: 1 │ +26)│ format: parquet │ +27)└───────────────────────────┘ # Query with projection on memory @@ -1201,18 +1105,12 @@ physical_plan 11)│ bigint_col │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ yield_stream_exec │ +14)│ DataSourceExec │ 15)│ -------------------- │ -16)│ YieldStreamExec child: │ -17)│ DataSourceExec │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ DataSourceExec │ -21)│ -------------------- │ -22)│ bytes: 1560 │ -23)│ format: memory │ -24)│ rows: 1 │ -25)└───────────────────────────┘ +16)│ bytes: 1560 │ +17)│ format: memory │ +18)│ rows: 1 │ +19)└───────────────────────────┘ # Query with projection on json query TT @@ -1240,17 +1138,11 @@ physical_plan 19)│ RoundRobinBatch(4) │ 20)└─────────────┬─────────────┘ 21)┌─────────────┴─────────────┐ -22)│ yield_stream_exec │ +22)│ DataSourceExec │ 23)│ -------------------- │ -24)│ YieldStreamExec child: │ -25)│ DataSourceExec │ -26)└─────────────┬─────────────┘ -27)┌─────────────┴─────────────┐ -28)│ DataSourceExec │ -29)│ -------------------- │ -30)│ files: 1 │ -31)│ format: json │ -32)└───────────────────────────┘ +24)│ files: 1 │ +25)│ format: json │ +26)└───────────────────────────┘ # Query with projection on arrow @@ -1280,17 +1172,11 @@ physical_plan 20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ -23)│ yield_stream_exec │ +23)│ DataSourceExec │ 24)│ -------------------- │ -25)│ YieldStreamExec child: │ -26)│ DataSourceExec │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ DataSourceExec │ -30)│ -------------------- │ -31)│ files: 1 │ -32)│ format: arrow │ -33)└───────────────────────────┘ +25)│ files: 1 │ +26)│ format: arrow │ +27)└───────────────────────────┘ # Query with PartialSortExec. query TT @@ -1307,17 +1193,11 @@ physical_plan 06)│ ASC NULLS LAST │ 07)└─────────────┬─────────────┘ 08)┌─────────────┴─────────────┐ -09)│ yield_stream_exec │ +09)│ StreamingTableExec │ 10)│ -------------------- │ -11)│ YieldStreamExec child: │ -12)│ StreamingTableExec │ -13)└─────────────┬─────────────┘ -14)┌─────────────┴─────────────┐ -15)│ StreamingTableExec │ -16)│ -------------------- │ -17)│ infinite: true │ -18)│ limit: None │ -19)└───────────────────────────┘ +11)│ infinite: true │ +12)│ limit: None │ +13)└───────────────────────────┘ query TT EXPLAIN SELECT * @@ -1336,17 +1216,11 @@ physical_plan 08)│ limit: 50 │ 09)└─────────────┬─────────────┘ 10)┌─────────────┴─────────────┐ -11)│ yield_stream_exec │ +11)│ StreamingTableExec │ 12)│ -------------------- │ -13)│ YieldStreamExec child: │ -14)│ StreamingTableExec │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ StreamingTableExec │ -18)│ -------------------- │ -19)│ infinite: true │ -20)│ limit: None │ -21)└───────────────────────────┘ +13)│ infinite: true │ +14)│ limit: None │ +15)└───────────────────────────┘ # Query with hash join. query TT @@ -1403,26 +1277,20 @@ physical_plan 48)│ string_col ││ │ 49)└─────────────┬─────────────┘└─────────────┬─────────────┘ 50)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -51)│ RepartitionExec ││ yield_stream_exec │ +51)│ RepartitionExec ││ DataSourceExec │ 52)│ -------------------- ││ -------------------- │ -53)│ partition_count(in->out): ││ YieldStreamExec child: │ -54)│ 1 -> 4 ││ DataSourceExec │ +53)│ partition_count(in->out): ││ files: 1 │ +54)│ 1 -> 4 ││ format: parquet │ 55)│ ││ │ 56)│ partitioning_scheme: ││ │ 57)│ RoundRobinBatch(4) ││ │ -58)└─────────────┬─────────────┘└─────────────┬─────────────┘ -59)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -60)│ yield_stream_exec ││ DataSourceExec │ -61)│ -------------------- ││ -------------------- │ -62)│ YieldStreamExec child: ││ files: 1 │ -63)│ DataSourceExec ││ format: parquet │ -64)└─────────────┬─────────────┘└───────────────────────────┘ -65)┌─────────────┴─────────────┐ -66)│ DataSourceExec │ -67)│ -------------------- │ -68)│ files: 1 │ -69)│ format: csv │ -70)└───────────────────────────┘ +58)└─────────────┬─────────────┘└───────────────────────────┘ +59)┌─────────────┴─────────────┐ +60)│ DataSourceExec │ +61)│ -------------------- │ +62)│ files: 1 │ +63)│ format: csv │ +64)└───────────────────────────┘ # Query with outer hash join. query TT @@ -1481,26 +1349,20 @@ physical_plan 50)│ string_col ││ │ 51)└─────────────┬─────────────┘└─────────────┬─────────────┘ 52)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -53)│ RepartitionExec ││ yield_stream_exec │ +53)│ RepartitionExec ││ DataSourceExec │ 54)│ -------------------- ││ -------------------- │ -55)│ partition_count(in->out): ││ YieldStreamExec child: │ -56)│ 1 -> 4 ││ DataSourceExec │ +55)│ partition_count(in->out): ││ files: 1 │ +56)│ 1 -> 4 ││ format: parquet │ 57)│ ││ │ 58)│ partitioning_scheme: ││ │ 59)│ RoundRobinBatch(4) ││ │ -60)└─────────────┬─────────────┘└─────────────┬─────────────┘ -61)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -62)│ yield_stream_exec ││ DataSourceExec │ -63)│ -------------------- ││ -------------------- │ -64)│ YieldStreamExec child: ││ files: 1 │ -65)│ DataSourceExec ││ format: parquet │ -66)└─────────────┬─────────────┘└───────────────────────────┘ -67)┌─────────────┴─────────────┐ -68)│ DataSourceExec │ -69)│ -------------------- │ -70)│ files: 1 │ -71)│ format: csv │ -72)└───────────────────────────┘ +60)└─────────────┬─────────────┘└───────────────────────────┘ +61)┌─────────────┴─────────────┐ +62)│ DataSourceExec │ +63)│ -------------------- │ +64)│ files: 1 │ +65)│ format: csv │ +66)└───────────────────────────┘ # Query with nested loop join. query TT @@ -1750,25 +1612,19 @@ physical_plan 02)│ UnionExec ├──────────────┐ 03)└─────────────┬─────────────┘ │ 04)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -05)│ yield_stream_exec ││ ProjectionExec │ +05)│ DataSourceExec ││ ProjectionExec │ 06)│ -------------------- ││ -------------------- │ -07)│ YieldStreamExec child: ││ id: CAST(id AS Int32) │ -08)│ DataSourceExec ││ name: name │ -09)└─────────────┬─────────────┘└─────────────┬─────────────┘ -10)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -11)│ DataSourceExec ││ yield_stream_exec │ -12)│ -------------------- ││ -------------------- │ -13)│ bytes: 1320 ││ YieldStreamExec child: │ -14)│ format: memory ││ DataSourceExec │ -15)│ rows: 1 ││ │ -16)└───────────────────────────┘└─────────────┬─────────────┘ -17)-----------------------------┌─────────────┴─────────────┐ -18)-----------------------------│ DataSourceExec │ -19)-----------------------------│ -------------------- │ -20)-----------------------------│ bytes: 1312 │ -21)-----------------------------│ format: memory │ -22)-----------------------------│ rows: 1 │ -23)-----------------------------└───────────────────────────┘ +07)│ bytes: 1320 ││ id: CAST(id AS Int32) │ +08)│ format: memory ││ name: name │ +09)│ rows: 1 ││ │ +10)└───────────────────────────┘└─────────────┬─────────────┘ +11)-----------------------------┌─────────────┴─────────────┐ +12)-----------------------------│ DataSourceExec │ +13)-----------------------------│ -------------------- │ +14)-----------------------------│ bytes: 1312 │ +15)-----------------------------│ format: memory │ +16)-----------------------------│ rows: 1 │ +17)-----------------------------└───────────────────────────┘ # cleanup statement ok @@ -1823,17 +1679,11 @@ physical_plan 25)│ RoundRobinBatch(4) │ 26)└─────────────┬─────────────┘ 27)┌─────────────┴─────────────┐ -28)│ yield_stream_exec │ +28)│ StreamingTableExec │ 29)│ -------------------- │ -30)│ YieldStreamExec child: │ -31)│ StreamingTableExec │ -32)└─────────────┬─────────────┘ -33)┌─────────────┴─────────────┐ -34)│ StreamingTableExec │ -35)│ -------------------- │ -36)│ infinite: true │ -37)│ limit: None │ -38)└───────────────────────────┘ +30)│ infinite: true │ +31)│ limit: None │ +32)└───────────────────────────┘ # constant ticker, CAST(time AS DATE) = time, order by time @@ -1871,17 +1721,11 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ yield_stream_exec │ +29)│ StreamingTableExec │ 30)│ -------------------- │ -31)│ YieldStreamExec child: │ -32)│ StreamingTableExec │ -33)└─────────────┬─────────────┘ -34)┌─────────────┴─────────────┐ -35)│ StreamingTableExec │ -36)│ -------------------- │ -37)│ infinite: true │ -38)│ limit: None │ -39)└───────────────────────────┘ +31)│ infinite: true │ +32)│ limit: None │ +33)└───────────────────────────┘ # same thing but order by date query TT @@ -1918,17 +1762,11 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ yield_stream_exec │ +29)│ StreamingTableExec │ 30)│ -------------------- │ -31)│ YieldStreamExec child: │ -32)│ StreamingTableExec │ -33)└─────────────┬─────────────┘ -34)┌─────────────┴─────────────┐ -35)│ StreamingTableExec │ -36)│ -------------------- │ -37)│ infinite: true │ -38)│ limit: None │ -39)└───────────────────────────┘ +31)│ infinite: true │ +32)│ limit: None │ +33)└───────────────────────────┘ # same thing but order by ticker query TT @@ -1963,17 +1801,11 @@ physical_plan 24)│ RoundRobinBatch(4) │ 25)└─────────────┬─────────────┘ 26)┌─────────────┴─────────────┐ -27)│ yield_stream_exec │ +27)│ StreamingTableExec │ 28)│ -------------------- │ -29)│ YieldStreamExec child: │ -30)│ StreamingTableExec │ -31)└─────────────┬─────────────┘ -32)┌─────────────┴─────────────┐ -33)│ StreamingTableExec │ -34)│ -------------------- │ -35)│ infinite: true │ -36)│ limit: None │ -37)└───────────────────────────┘ +29)│ infinite: true │ +30)│ limit: None │ +31)└───────────────────────────┘ # same thing but order by time, date @@ -2012,17 +1844,11 @@ physical_plan 27)│ RoundRobinBatch(4) │ 28)└─────────────┬─────────────┘ 29)┌─────────────┴─────────────┐ -30)│ yield_stream_exec │ +30)│ StreamingTableExec │ 31)│ -------------------- │ -32)│ YieldStreamExec child: │ -33)│ StreamingTableExec │ -34)└─────────────┬─────────────┘ -35)┌─────────────┴─────────────┐ -36)│ StreamingTableExec │ -37)│ -------------------- │ -38)│ infinite: true │ -39)│ limit: None │ -40)└───────────────────────────┘ +32)│ infinite: true │ +33)│ limit: None │ +34)└───────────────────────────┘ @@ -2062,17 +1888,11 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ yield_stream_exec │ +29)│ StreamingTableExec │ 30)│ -------------------- │ -31)│ YieldStreamExec child: │ -32)│ StreamingTableExec │ -33)└─────────────┬─────────────┘ -34)┌─────────────┴─────────────┐ -35)│ StreamingTableExec │ -36)│ -------------------- │ -37)│ infinite: true │ -38)│ limit: None │ -39)└───────────────────────────┘ +31)│ infinite: true │ +32)│ limit: None │ +33)└───────────────────────────┘ @@ -2097,42 +1917,35 @@ physical_plan 07)│ id: 1 ││ │ 08)└─────────────┬─────────────┘└─────────────┬─────────────┘ 09)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -10)│ yield_stream_exec ││ ProjectionExec │ -11)│ -------------------- ││ -------------------- │ -12)│ YieldStreamExec child: ││ id: id + 1 │ -13)│ PlaceholderRowExec ││ │ -14)└─────────────┬─────────────┘└─────────────┬─────────────┘ -15)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -16)│ PlaceholderRowExec ││ CoalesceBatchesExec │ -17)│ ││ -------------------- │ -18)│ ││ target_batch_size: │ -19)│ ││ 8192 │ -20)└───────────────────────────┘└─────────────┬─────────────┘ -21)-----------------------------┌─────────────┴─────────────┐ -22)-----------------------------│ FilterExec │ -23)-----------------------------│ -------------------- │ -24)-----------------------------│ predicate: id < 10 │ -25)-----------------------------└─────────────┬─────────────┘ -26)-----------------------------┌─────────────┴─────────────┐ -27)-----------------------------│ RepartitionExec │ -28)-----------------------------│ -------------------- │ -29)-----------------------------│ partition_count(in->out): │ -30)-----------------------------│ 1 -> 4 │ -31)-----------------------------│ │ -32)-----------------------------│ partitioning_scheme: │ -33)-----------------------------│ RoundRobinBatch(4) │ -34)-----------------------------└─────────────┬─────────────┘ -35)-----------------------------┌─────────────┴─────────────┐ -36)-----------------------------│ yield_stream_exec │ -37)-----------------------------│ -------------------- │ -38)-----------------------------│ YieldStreamExec child: │ -39)-----------------------------│ WorkTableExec │ -40)-----------------------------└─────────────┬─────────────┘ -41)-----------------------------┌─────────────┴─────────────┐ -42)-----------------------------│ WorkTableExec │ -43)-----------------------------│ -------------------- │ -44)-----------------------------│ name: nodes │ -45)-----------------------------└───────────────────────────┘ +10)│ PlaceholderRowExec ││ ProjectionExec │ +11)│ ││ -------------------- │ +12)│ ││ id: id + 1 │ +13)└───────────────────────────┘└─────────────┬─────────────┘ +14)-----------------------------┌─────────────┴─────────────┐ +15)-----------------------------│ CoalesceBatchesExec │ +16)-----------------------------│ -------------------- │ +17)-----------------------------│ target_batch_size: │ +18)-----------------------------│ 8192 │ +19)-----------------------------└─────────────┬─────────────┘ +20)-----------------------------┌─────────────┴─────────────┐ +21)-----------------------------│ FilterExec │ +22)-----------------------------│ -------------------- │ +23)-----------------------------│ predicate: id < 10 │ +24)-----------------------------└─────────────┬─────────────┘ +25)-----------------------------┌─────────────┴─────────────┐ +26)-----------------------------│ RepartitionExec │ +27)-----------------------------│ -------------------- │ +28)-----------------------------│ partition_count(in->out): │ +29)-----------------------------│ 1 -> 4 │ +30)-----------------------------│ │ +31)-----------------------------│ partitioning_scheme: │ +32)-----------------------------│ RoundRobinBatch(4) │ +33)-----------------------------└─────────────┬─────────────┘ +34)-----------------------------┌─────────────┴─────────────┐ +35)-----------------------------│ WorkTableExec │ +36)-----------------------------│ -------------------- │ +37)-----------------------------│ name: nodes │ +38)-----------------------------└───────────────────────────┘ query TT explain COPY (VALUES (1, 'foo', 1, '2023-01-01'), (2, 'bar', 2, '2023-01-02'), (3, 'baz', 3, '2023-01-03')) @@ -2150,18 +1963,12 @@ physical_plan 09)│ format: json │ 10)└─────────────┬─────────────┘ 11)┌─────────────┴─────────────┐ -12)│ yield_stream_exec │ +12)│ DataSourceExec │ 13)│ -------------------- │ -14)│ YieldStreamExec child: │ -15)│ DataSourceExec │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ DataSourceExec │ -19)│ -------------------- │ -20)│ bytes: 2672 │ -21)│ format: memory │ -22)│ rows: 1 │ -23)└───────────────────────────┘ +14)│ bytes: 2672 │ +15)│ format: memory │ +16)│ rows: 1 │ +17)└───────────────────────────┘ query TT explain COPY (VALUES (1, 'foo', 1, '2023-01-01'), (2, 'bar', 2, '2023-01-02'), (3, 'baz', 3, '2023-01-03')) @@ -2179,18 +1986,12 @@ physical_plan 09)│ format: csv │ 10)└─────────────┬─────────────┘ 11)┌─────────────┴─────────────┐ -12)│ yield_stream_exec │ +12)│ DataSourceExec │ 13)│ -------------------- │ -14)│ YieldStreamExec child: │ -15)│ DataSourceExec │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ DataSourceExec │ -19)│ -------------------- │ -20)│ bytes: 2672 │ -21)│ format: memory │ -22)│ rows: 1 │ -23)└───────────────────────────┘ +14)│ bytes: 2672 │ +15)│ format: memory │ +16)│ rows: 1 │ +17)└───────────────────────────┘ query TT explain COPY (VALUES (1, 'foo', 1, '2023-01-01'), (2, 'bar', 2, '2023-01-02'), (3, 'baz', 3, '2023-01-03')) @@ -2208,18 +2009,12 @@ physical_plan 09)│ format: arrow │ 10)└─────────────┬─────────────┘ 11)┌─────────────┴─────────────┐ -12)│ yield_stream_exec │ +12)│ DataSourceExec │ 13)│ -------------------- │ -14)│ YieldStreamExec child: │ -15)│ DataSourceExec │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ DataSourceExec │ -19)│ -------------------- │ -20)│ bytes: 2672 │ -21)│ format: memory │ -22)│ rows: 1 │ -23)└───────────────────────────┘ +14)│ bytes: 2672 │ +15)│ format: memory │ +16)│ rows: 1 │ +17)└───────────────────────────┘ # Test explain tree rendering for CoalesceBatchesExec with limit @@ -2306,18 +2101,12 @@ EXPLAIN SELECT * FROM generate_series(1, 100) ---- physical_plan 01)┌───────────────────────────┐ -02)│ yield_stream_exec │ +02)│ LazyMemoryExec │ 03)│ -------------------- │ -04)│ YieldStreamExec child: │ -05)│ LazyMemoryExec │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ LazyMemoryExec │ -09)│ -------------------- │ -10)│ batch_generators: │ -11)│ generate_series: start=1, │ -12)│ end=100, batch_size=8192 │ -13)└───────────────────────────┘ +04)│ batch_generators: │ +05)│ generate_series: start=1, │ +06)│ end=100, batch_size=8192 │ +07)└───────────────────────────┘ # Test explain tree for CoalescePartitionsExec query TT @@ -2352,17 +2141,11 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ yield_stream_exec │ +29)│ StreamingTableExec │ 30)│ -------------------- │ -31)│ YieldStreamExec child: │ -32)│ StreamingTableExec │ -33)└─────────────┬─────────────┘ -34)┌─────────────┴─────────────┐ -35)│ StreamingTableExec │ -36)│ -------------------- │ -37)│ infinite: true │ -38)│ limit: None │ -39)└───────────────────────────┘ +31)│ infinite: true │ +32)│ limit: None │ +33)└───────────────────────────┘ # Test explain tree for PlaceholderRowExec query TT @@ -2375,11 +2158,5 @@ physical_plan 04)│ count(*): 2 │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ yield_stream_exec │ -08)│ -------------------- │ -09)│ YieldStreamExec child: │ -10)│ PlaceholderRowExec │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ PlaceholderRowExec │ -14)└───────────────────────────┘ +07)│ PlaceholderRowExec │ +08)└───────────────────────────┘ diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index 749e38c1a44a..a09d8ce26ddf 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -41,8 +41,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------YieldStreamExec child=StreamingTableExec -06)----------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] +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 @@ -59,8 +58,7 @@ physical_plan 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)--------YieldStreamExec child=StreamingTableExec -06)----------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] +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 @@ -77,8 +75,7 @@ physical_plan 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)--------YieldStreamExec child=StreamingTableExec -06)----------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] +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 @@ -95,8 +92,7 @@ physical_plan 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)--------YieldStreamExec child=StreamingTableExec -06)----------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] +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 @@ -113,8 +109,7 @@ physical_plan 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)--------YieldStreamExec child=StreamingTableExec -06)----------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] +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 @@ -155,5 +150,4 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: date@0 = 2006-01-02 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------YieldStreamExec child=StreamingTableExec -06)----------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] +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 df68c1a16104..f9deb3d69b16 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2084,9 +2084,7 @@ logical_plan 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 -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # Final plan shouldn't have SortExec a ASC, b ASC, # because table already satisfies this ordering. @@ -2097,9 +2095,7 @@ logical_plan 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 -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # test_window_agg_sort statement ok @@ -2119,8 +2115,7 @@ logical_plan physical_plan 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)----YieldStreamExec child=StreamingTableExec -04)------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] +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 @@ -2151,8 +2146,7 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=PartiallySorted([1]) -03)----YieldStreamExec child=StreamingTableExec -04)------StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] +03)----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] query III SELECT a, d, @@ -2185,8 +2179,7 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted -03)----YieldStreamExec child=StreamingTableExec -04)------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] +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 @@ -2212,8 +2205,7 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted -03)----YieldStreamExec child=StreamingTableExec -04)------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] +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 @@ -2240,8 +2232,7 @@ logical_plan physical_plan 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)----YieldStreamExec child=StreamingTableExec -04)------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] +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 c) as last_c @@ -2263,8 +2254,7 @@ logical_plan 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan 01)PartialSortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], common_prefix_length=[2] -02)--YieldStreamExec child=StreamingTableExec -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] +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 * @@ -2277,8 +2267,7 @@ logical_plan 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan 01)PartialSortExec: TopK(fetch=50), expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], common_prefix_length=[2] -02)--YieldStreamExec child=StreamingTableExec -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] +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 * @@ -2304,8 +2293,7 @@ logical_plan physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[array_agg(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]], 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)----YieldStreamExec child=StreamingTableExec -04)------StreamingTableExec: partition_sizes=1, projection=[a, b, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST] +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? @@ -3949,9 +3937,7 @@ ORDER BY c ASC; logical_plan 01)Sort: multiple_ordered_table.c ASC NULLS LAST 02)--TableScan: multiple_ordered_table projection=[c] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -3985,12 +3971,10 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], 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)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true -07)--------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] -08)----------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] -09)------------YieldStreamExec child=DataSourceExec -10)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, 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)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # reset partition number to 8. statement ok @@ -4096,8 +4080,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT c, sum1, SUM(b) OVER() as sumb @@ -4149,12 +4132,10 @@ physical_plan 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)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true -08)------ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] -09)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -10)----------YieldStreamExec child=DataSourceExec -11)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, 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)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4227,12 +4208,10 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true -06)--ProjectionExec: expr=[c@0 as c, a@1 as a, sum(multiple_ordered_table_with_pk.d)@2 as sum1] -07)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -08)------YieldStreamExec child=DataSourceExec -09)--------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +04)------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, 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)------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # table scan should be simplified. query TT @@ -4247,8 +4226,7 @@ logical_plan physical_plan 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)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +03)----DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # limit should be simplified query TT @@ -4267,8 +4245,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +04)------DataSourceExec: 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]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 8; @@ -4362,8 +4339,7 @@ physical_plan 05)--------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 06)----------AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------YieldStreamExec child=StreamingTableExec -09)----------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] +08)--------------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 @@ -4459,8 +4435,7 @@ physical_plan 01)SortPreservingMergeExec: [name@0 DESC, time_chunks@1 DESC], fetch=5 02)--ProjectionExec: expr=[name@0 as name, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@1) as time_chunks] 03)----RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -04)------YieldStreamExec child=StreamingTableExec -05)--------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] +04)------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] statement ok drop table t1 @@ -5100,8 +5075,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, b, c] physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[array_agg(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]], ordering_mode=Sorted -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: 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]], file_type=csv, has_header=true +02)--DataSourceExec: 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]], file_type=csv, has_header=true query II? SELECT a, b, ARRAY_AGG(c ORDER BY c DESC) diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 41ea1cdcd922..6b3e3f28f903 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -682,9 +682,7 @@ query TT explain select * from t1 join t2 on false; ---- logical_plan EmptyRelation -physical_plan -01)YieldStreamExec child=EmptyExec -02)--EmptyExec +physical_plan EmptyExec # Make batch size smaller than table row number. to introduce parallelism to the plan. statement ok @@ -782,10 +780,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] # Reset the configs to old values statement ok @@ -856,10 +852,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[emp_id@1 as emp_id, name@2 as name, dept_name@0 as dept_name] 02)--NestedLoopJoinExec: join_type=Right, filter=name@0 = Alice OR name@0 = Bob -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] query ITT SELECT e.emp_id, e.name, d.dept_name @@ -1000,10 +994,8 @@ physical_plan 05)--------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)----------YieldStreamExec child=DataSourceExec -11)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------DataSourceExec: partitions=1, partition_sizes=[1] query ITT SELECT e.emp_id, e.name, d.dept_name @@ -1198,12 +1190,9 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0), (v1@1, v1@1)], projection=[v0@0, v1@1, v0@2, v2@4, v3@5, v4@6] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[0] -10)--------------YieldStreamExec child=DataSourceExec -11)----------------DataSourceExec: partitions=1, partition_sizes=[0] -12)----YieldStreamExec child=DataSourceExec -13)------DataSourceExec: partitions=1, partition_sizes=[0] +08)--------------DataSourceExec: partitions=1, partition_sizes=[0] +09)--------------DataSourceExec: partitions=1, partition_sizes=[0] +10)----DataSourceExec: partitions=1, partition_sizes=[0] @@ -1392,20 +1381,17 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(s.b AS Int64)@1, col1@1)], projection=[b@0, col0@2, col1@3, a@4] 04)------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)------ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] -08)--------CoalesceBatchesExec: target_batch_size=8192 -09)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] -10)------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] -11)--------------YieldStreamExec child=DataSourceExec -12)----------------DataSourceExec: partitions=1, partition_sizes=[1] -13)------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] -14)--------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 -15)----------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------FilterExec: y@1 = x@0 -17)--------------------YieldStreamExec child=DataSourceExec -18)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] +07)--------CoalesceBatchesExec: target_batch_size=8192 +08)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] +09)------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] +10)--------------DataSourceExec: partitions=1, partition_sizes=[1] +11)------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] +12)--------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 +13)----------------CoalesceBatchesExec: target_batch_size=8192 +14)------------------FilterExec: y@1 = x@0 +15)--------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table pairs; @@ -1460,8 +1446,7 @@ physical_plan 10)------------------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] 11)--------------------YieldStreamExec child=DataSourceExec 12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] -13)------YieldStreamExec child=DataSourceExec -14)--------DataSourceExec: partitions=1, partition_sizes=[1] +13)------DataSourceExec: partitions=1, partition_sizes=[1] query III SELECT * @@ -1483,10 +1468,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] query IIII SELECT * FROM t0, LATERAL (SELECT * FROM t1 WHERE t0.v0 = t1.v0); diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 7633ea77f9df..a1efc1317b4a 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -57,11 +57,9 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], file_type=csv, has_header=true -06)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)--------YieldStreamExec child=DataSourceExec -08)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], file_type=csv, has_header=true +05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # preserve_inner_join query IIII nosort @@ -100,13 +98,11 @@ physical_plan 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true -07)--------CoalesceBatchesExec: target_batch_size=8192 -08)----------FilterExec: d@3 = 3 -09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)--------------YieldStreamExec child=DataSourceExec -11)----------------DataSourceExec: 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], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true +06)--------CoalesceBatchesExec: target_batch_size=8192 +07)----------FilterExec: d@3 = 3 +08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)--------------DataSourceExec: 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], file_type=csv, 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 a3c4429ed93b..8d368eba43be 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1363,11 +1363,9 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s3@0, s4@0)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)------YieldStreamExec child=DataSourceExec -07)--------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)------DataSourceExec: partitions=1, partition_sizes=[1] query ?? select join_t3.s3, join_t4.s4 @@ -1505,12 +1503,10 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------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)] -10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)----------YieldStreamExec child=DataSourceExec -12)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1533,12 +1529,10 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------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)] -10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)----------YieldStreamExec child=DataSourceExec -12)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1563,12 +1557,10 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------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)] -10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)----------YieldStreamExec child=DataSourceExec -12)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1591,12 +1583,10 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------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)] -10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)----------YieldStreamExec child=DataSourceExec -12)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1619,12 +1609,10 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------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)] -07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1645,12 +1633,10 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------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)] -07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: 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)----------DataSourceExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1676,11 +1662,9 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)--------YieldStreamExec child=DataSourceExec -11)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1704,11 +1688,9 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)--------YieldStreamExec child=DataSourceExec -11)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -1729,12 +1711,10 @@ logical_plan physical_plan 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)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----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)] -06)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)--------YieldStreamExec child=DataSourceExec -08)----------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: 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)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1753,12 +1733,10 @@ logical_plan physical_plan 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)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----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)] -06)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)--------YieldStreamExec child=DataSourceExec -08)----------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: 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)--------DataSourceExec: partitions=1, partition_sizes=[1] ##### # Config teardown @@ -2083,13 +2061,11 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)----CoalesceBatchesExec: target_batch_size=2 -10)------FilterExec: t1_id@0 > 10 -11)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------YieldStreamExec child=DataSourceExec -13)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)----CoalesceBatchesExec: target_batch_size=2 +09)------FilterExec: t1_id@0 > 10 +10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)----------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2124,13 +2100,11 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------FilterExec: t1_id@0 > 22 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)--CoalesceBatchesExec: target_batch_size=2 -09)----FilterExec: t2_id@0 > 11 -10)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------YieldStreamExec child=DataSourceExec -12)----------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: 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)--------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2601,11 +2575,9 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(millis@2, millis@2)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)------YieldStreamExec child=DataSourceExec -07)--------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)------DataSourceExec: partitions=1, partition_sizes=[1] # left_join_using_2 query II @@ -3250,13 +3222,11 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------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] 07)------------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] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: 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], file_type=csv, has_header=true -10)----CoalesceBatchesExec: target_batch_size=2 -11)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST -12)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)----------YieldStreamExec child=DataSourceExec -14)------------DataSourceExec: 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], file_type=csv, has_header=true +08)--------------DataSourceExec: 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], file_type=csv, has_header=true +09)----CoalesceBatchesExec: target_batch_size=2 +10)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST +11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------DataSourceExec: 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], file_type=csv, 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 @@ -3284,15 +3254,13 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: 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], file_type=csv, has_header=true -08)----CoalesceBatchesExec: target_batch_size=2 -09)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST -10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)----------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] -12)------------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] -13)--------------YieldStreamExec child=DataSourceExec -14)----------------DataSourceExec: 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], file_type=csv, has_header=true +06)----------DataSourceExec: 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], file_type=csv, has_header=true +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------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] +11)------------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] +12)--------------DataSourceExec: 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], file_type=csv, has_header=true statement ok set datafusion.optimizer.prefer_existing_sort = false; @@ -3372,12 +3340,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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], file_type=csv, has_header=true -05)----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] -06)------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] -07)--------YieldStreamExec child=DataSourceExec -08)----------DataSourceExec: 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], file_type=csv, has_header=true +03)----DataSourceExec: 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], file_type=csv, 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)--------DataSourceExec: 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], file_type=csv, 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. @@ -3401,12 +3367,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], file_type=csv, has_header=true -05)----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] -06)------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] -07)--------YieldStreamExec child=DataSourceExec -08)----------DataSourceExec: 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], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], file_type=csv, 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)--------DataSourceExec: 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], file_type=csv, has_header=true query TT EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 @@ -3430,10 +3394,8 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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], file_type=csv, has_header=true -07)--------YieldStreamExec child=DataSourceExec -08)----------DataSourceExec: 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], file_type=csv, has_header=true +05)--------DataSourceExec: 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], file_type=csv, has_header=true +06)--------DataSourceExec: 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], file_type=csv, has_header=true # create a table where there more than one valid ordering # that describes table. @@ -3480,12 +3442,10 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], 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)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true -07)--------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] -08)----------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] -09)------------YieldStreamExec child=DataSourceExec -10)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, 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)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # run query above in multiple partitions statement ok @@ -3546,11 +3506,9 @@ logical_plan 05)----TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=a@1 < a@0 -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: 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], file_type=csv, has_header=true -04)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: 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], file_type=csv, has_header=true +02)--DataSourceExec: 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], file_type=csv, has_header=true +03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)----DataSourceExec: 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], file_type=csv, has_header=true # Currently datafusion can pushdown filter conditions with scalar UDF into # cross join. @@ -3567,11 +3525,9 @@ logical_plan 05)----TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=example(CAST(a@0 AS Float64), CAST(a@1 AS Float64)) > 3 -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: 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], file_type=csv, has_header=true -04)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: 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], file_type=csv, has_header=true +02)--DataSourceExec: 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], file_type=csv, has_header=true +03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)----DataSourceExec: 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], file_type=csv, has_header=true #### # Config teardown @@ -3689,15 +3645,12 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] 04)------ProjectionExec: expr=[1 as e, 3 as f] -05)--------YieldStreamExec child=PlaceholderRowExec -06)----------PlaceholderRowExec -07)------UnionExec -08)--------ProjectionExec: expr=[1 as c, 2 as d] -09)----------YieldStreamExec child=PlaceholderRowExec -10)------------PlaceholderRowExec -11)--------ProjectionExec: expr=[1 as c, 3 as d] -12)----------YieldStreamExec child=PlaceholderRowExec -13)------------PlaceholderRowExec +05)--------PlaceholderRowExec +06)------UnionExec +07)--------ProjectionExec: expr=[1 as c, 2 as d] +08)----------PlaceholderRowExec +09)--------ProjectionExec: expr=[1 as c, 3 as d] +10)----------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3735,15 +3688,12 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] 04)------ProjectionExec: expr=[1 as e, 3 as f] -05)--------YieldStreamExec child=PlaceholderRowExec -06)----------PlaceholderRowExec -07)------UnionExec -08)--------ProjectionExec: expr=[1 as c, 2 as d] -09)----------YieldStreamExec child=PlaceholderRowExec -10)------------PlaceholderRowExec -11)--------ProjectionExec: expr=[1 as c, 3 as d] -12)----------YieldStreamExec child=PlaceholderRowExec -13)------------PlaceholderRowExec +05)--------PlaceholderRowExec +06)------UnionExec +07)--------ProjectionExec: expr=[1 as c, 2 as d] +08)----------PlaceholderRowExec +09)--------ProjectionExec: expr=[1 as c, 3 as d] +10)----------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3953,8 +3903,7 @@ physical_plan 04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] 05)--------YieldStreamExec child=DataSourceExec 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)------YieldStreamExec child=DataSourceExec -08)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)------DataSourceExec: partitions=1, partition_sizes=[1] @@ -4010,10 +3959,8 @@ physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------YieldStreamExec child=DataSourceExec -07)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------DataSourceExec: partitions=1, partition_sizes=[1] # Null build indices: @@ -4075,8 +4022,7 @@ physical_plan 04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] 05)--------YieldStreamExec child=DataSourceExec 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)------YieldStreamExec child=DataSourceExec -08)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)------DataSourceExec: partitions=1, partition_sizes=[1] # Test CROSS JOIN LATERAL syntax (planning) @@ -4282,10 +4228,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] ## Test join.on.is_empty() && join.filter.is_some() query TT @@ -4299,10 +4243,8 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=0, fetch=2 02)--NestedLoopJoinExec: join_type=Full, filter=c2@0 >= c2@1 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] ## Test !join.on.is_empty() && join.filter.is_some() query TT @@ -4316,10 +4258,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)], filter=c2@0 >= c2@1 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] ## Add more test cases for join limit pushdown statement ok @@ -4382,10 +4322,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, b@0)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], limit=2, file_type=csv, has_header=true -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], limit=2, file_type=csv, has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true ###### ## RIGHT JOIN w/ LIMIT @@ -4418,10 +4356,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@0, b@0)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true ###### ## FULL JOIN w/ LIMIT @@ -4457,10 +4393,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(a@0, b@0)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true statement ok drop table t1; @@ -4607,10 +4541,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2, state@5] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT age FROM (SELECT * FROM person a join person b using (id, age, state)); @@ -4625,10 +4557,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[age@1] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT a.* FROM person a join person b using (id, age); @@ -4643,10 +4573,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT a.*, b.* FROM person a join person b using (id, age); @@ -4660,10 +4588,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT * FROM person a join person b using (id, age, state) join person c using (id, age, state); @@ -4684,12 +4610,9 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] 03)----CoalesceBatchesExec: target_batch_size=3 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: partitions=1, partition_sizes=[0] -07)--------YieldStreamExec child=DataSourceExec -08)----------DataSourceExec: partitions=1, partition_sizes=[0] -09)----YieldStreamExec child=DataSourceExec -10)------DataSourceExec: partitions=1, partition_sizes=[0] +05)--------DataSourceExec: partitions=1, partition_sizes=[0] +06)--------DataSourceExec: partitions=1, partition_sizes=[0] +07)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT * FROM person a NATURAL JOIN lineitem b; @@ -4719,10 +4642,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c1@0, c1@0)], projection=[c1@0] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; @@ -4860,11 +4781,9 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(person.id + Int64(10)@1, orders.customer_id * Int64(2)@1)], projection=[id@0, customer_id@2] 03)----ProjectionExec: expr=[id@0 as id, CAST(id@0 AS Int64) + 10 as person.id + Int64(10)] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[0] -06)----ProjectionExec: expr=[customer_id@0 as customer_id, CAST(customer_id@0 AS Int64) * 2 as orders.customer_id * Int64(2)] -07)------YieldStreamExec child=DataSourceExec -08)--------DataSourceExec: partitions=1, partition_sizes=[0] +04)------DataSourceExec: partitions=1, partition_sizes=[0] +05)----ProjectionExec: expr=[customer_id@0 as customer_id, CAST(customer_id@0 AS Int64) * 2 as orders.customer_id * Int64(2)] +06)------DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index 1c642b654439..ea243916c291 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -146,6 +146,4 @@ query TT EXPLAIN SELECT id FROM json_partitioned_test WHERE part = 2 ---- logical_plan TableScan: json_partitioned_test projection=[id], full_filters=[json_partitioned_test.part = Int32(2)] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_json/part=2/data.json]]}, projection=[id], file_type=json +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_json/part=2/data.json]]}, projection=[id], file_type=json diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index b898faf76c7c..ae178f9feaaa 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -313,8 +313,7 @@ logical_plan 04)------TableScan: t1 projection=[], fetch=14 physical_plan 01)ProjectionExec: expr=[0 as count(*)] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 11); @@ -333,8 +332,7 @@ logical_plan 04)------TableScan: t1 projection=[], fetch=11 physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -353,8 +351,7 @@ logical_plan 04)------TableScan: t1 projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--YieldStreamExec child=PlaceholderRowExec -03)----PlaceholderRowExec +02)--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -557,9 +554,7 @@ explain SELECT * FROM data LIMIT 3; logical_plan 01)Limit: skip=0, fetch=3 02)--TableScan: data projection=[column1, column2], fetch=3 -physical_plan -01)YieldStreamExec child=StreamingTableExec -02)--StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=3, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] +physical_plan StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=3, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] # Do not remove limit with Sort when skip is used @@ -572,8 +567,7 @@ logical_plan 03)----TableScan: data projection=[column1, column2] physical_plan 01)GlobalLimitExec: skip=3, fetch=3 -02)--YieldStreamExec child=StreamingTableExec -03)----StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=6, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] +02)--StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=6, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] statement ok diff --git a/datafusion/sqllogictest/test_files/listing_table_statistics.slt b/datafusion/sqllogictest/test_files/listing_table_statistics.slt index f7add64610a4..890d1f2e9250 100644 --- a/datafusion/sqllogictest/test_files/listing_table_statistics.slt +++ b/datafusion/sqllogictest/test_files/listing_table_statistics.slt @@ -35,9 +35,7 @@ query TT explain format indent select * from t; ---- logical_plan TableScan: t projection=[int_col, str_col] -physical_plan -01)YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:)]] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/2.parquet]]}, projection=[int_col, str_col], file_type=parquet, statistics=[Rows=Exact(4), Bytes=Exact(288), [(Col[0]: Min=Exact(Int64(-1)) Max=Exact(Int64(3)) Null=Exact(0)),(Col[1]: Min=Exact(Utf8View("a")) Max=Exact(Utf8View("d")) Null=Exact(0))]] +physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/2.parquet]]}, projection=[int_col, str_col], file_type=parquet, statistics=[Rows=Exact(4), Bytes=Exact(288), [(Col[0]: Min=Exact(Int64(-1)) Max=Exact(Int64(3)) Null=Exact(0)),(Col[1]: Min=Exact(Utf8View("a")) Max=Exact(Utf8View("d")) Null=Exact(0))]] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 5fd3c3a6436d..42a4ba621801 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -115,8 +115,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int_field@0 > 0 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: 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 60e10e6e0012..d18b151dffea 100644 --- a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt +++ b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt @@ -47,8 +47,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: 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], file_type=csv, has_header=true +04)------DataSourceExec: 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], file_type=csv, has_header=true query TT EXPLAIN @@ -64,8 +63,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: 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], file_type=csv, has_header=true +04)------DataSourceExec: 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], file_type=csv, has_header=true # Cast to larger types as well as preserving ordering # doesn't invalidate lexicographical ordering. @@ -86,8 +84,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: 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], file_type=csv, has_header=true +04)------DataSourceExec: 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], file_type=csv, has_header=true # test for common rename query TT @@ -102,8 +99,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, a@0 as a_big, b@1 as b] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: 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], file_type=csv, has_header=true +02)--DataSourceExec: 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], file_type=csv, has_header=true query TT EXPLAIN @@ -117,8 +113,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, a@0 as a_big, b@1 as b] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: 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], file_type=csv, has_header=true +02)--DataSourceExec: 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], file_type=csv, has_header=true # test for cast Utf8 diff --git a/datafusion/sqllogictest/test_files/operator.slt b/datafusion/sqllogictest/test_files/operator.slt index 051b81c3efe8..a651eda99684 100644 --- a/datafusion/sqllogictest/test_files/operator.slt +++ b/datafusion/sqllogictest/test_files/operator.slt @@ -280,8 +280,7 @@ WHERE int64 < 5 AND uint64 < 5 AND float64 < 5 AND decimal < 5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 < 5 AND uint64@7 < 5 AND float64@9 < 5 AND decimal@10 < Some(500),5,2 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## < negative integer (expect no casts) query TT @@ -291,8 +290,7 @@ WHERE int64 < -5 AND uint64 < -5 AND float64 < -5 AND decimal < -5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 < -5 AND CAST(uint64@7 AS Decimal128(20, 0)) < Some(-5),20,0 AND float64@9 < -5 AND decimal@10 < Some(-500),5,2 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## < decimal (expect casts for integers to float) query TT @@ -302,8 +300,7 @@ WHERE int64 < 5.1 AND uint64 < 5.1 AND float64 < 5.1 AND decimal < 5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) < 5.1 AND CAST(uint64@7 AS Float64) < 5.1 AND float64@9 < 5.1 AND decimal@10 < Some(510),5,2 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## < negative decimal (expect casts for integers to float) query TT @@ -313,8 +310,7 @@ WHERE int64 < -5.1 AND uint64 < -5.1 AND float64 < -5.1 AND decimal < -5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) < -5.1 AND CAST(uint64@7 AS Float64) < -5.1 AND float64@9 < -5.1 AND decimal@10 < Some(-510),5,2 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ############### Equality ############### @@ -327,8 +323,7 @@ WHERE int64 = 5 AND uint64 = 5 AND float64 = 5 AND decimal = 5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 = 5 AND uint64@7 = 5 AND float64@9 = 5 AND decimal@10 = Some(500),5,2 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## = negative integer (expect no casts) query TT @@ -338,8 +333,7 @@ WHERE int64 = -5 AND uint64 = -5 AND float64 = -5 AND decimal = -5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 = -5 AND CAST(uint64@7 AS Decimal128(20, 0)) = Some(-5),20,0 AND float64@9 = -5 AND decimal@10 = Some(-500),5,2 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## = decimal (expect casts for integers to float) query TT @@ -349,8 +343,7 @@ WHERE int64 = 5.1 AND uint64 = 5.1 AND float64 = 5.1 AND decimal = 5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) = 5.1 AND CAST(uint64@7 AS Float64) = 5.1 AND float64@9 = 5.1 AND decimal@10 = Some(510),5,2 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## = negative decimal (expect casts for integers to float) query TT @@ -360,8 +353,7 @@ WHERE int64 = -5.1 AND uint64 = -5.1 AND float64 = -5.1 AND decimal = -5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) = -5.1 AND CAST(uint64@7 AS Float64) = -5.1 AND float64@9 = -5.1 AND decimal@10 = Some(-510),5,2 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/options.slt b/datafusion/sqllogictest/test_files/options.slt index 04d52e7bc953..71ff12e8cc50 100644 --- a/datafusion/sqllogictest/test_files/options.slt +++ b/datafusion/sqllogictest/test_files/options.slt @@ -33,8 +33,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: c0@0 < 1 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## # test_disable_coalesce @@ -52,8 +51,7 @@ logical_plan 02)--TableScan: a projection=[c0] physical_plan 01)FilterExec: c0@0 < 1 -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.execution.coalesce_batches = true @@ -76,8 +74,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=1234 02)--FilterExec: c0@0 < 1 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 56a1bbd68f71..6aed9bfd38d7 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -471,8 +471,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: 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]], file_type=csv, has_header=true +04)------DataSourceExec: 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]], file_type=csv, has_header=true statement ok drop table multiple_ordered_table; @@ -503,8 +502,7 @@ physical_plan 01)SortPreservingMergeExec: [db15@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@0, 1659537600000000000) as db15] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=false +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=false query TT EXPLAIN SELECT DATE_TRUNC('DAY', ts) as dt_day @@ -519,8 +517,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=false +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=false statement ok drop table csv_with_timestamps; @@ -563,8 +560,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT CEIL(c11) as ceil_c11 @@ -579,8 +575,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT LOG(c12, c11) as log_c11_base_c12 @@ -595,8 +590,7 @@ physical_plan 01)SortPreservingMergeExec: [log_c11_base_c12@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[log(c12@1, CAST(c11@0 AS Float64)) as log_c11_base_c12] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: 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 NULLS LAST]], file_type=csv, has_header=true +04)------DataSourceExec: 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 NULLS LAST]], file_type=csv, has_header=true query TT EXPLAIN SELECT LOG(c11, c12) as log_c12_base_c11 @@ -611,8 +605,7 @@ physical_plan 01)SortPreservingMergeExec: [log_c12_base_c11@0 DESC NULLS LAST] 02)--ProjectionExec: expr=[log(CAST(c11@0 AS Float64), c12@1) as log_c12_base_c11] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: 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 NULLS LAST]], file_type=csv, has_header=true +04)------DataSourceExec: 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 NULLS LAST]], file_type=csv, has_header=true statement ok drop table aggregate_test_100; @@ -674,9 +667,7 @@ EXPLAIN SELECT o_orderkey, o_orderstatus FROM orders ORDER BY o_orderkey ASC logical_plan 01)Sort: orders.o_orderkey ASC NULLS LAST 02)--TableScan: orders projection=[o_orderkey, o_orderstatus] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: 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], file_type=csv, has_header=true +physical_plan DataSourceExec: 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], file_type=csv, has_header=true # Create external table with DDL ordered columns that are missing @@ -1092,8 +1083,7 @@ physical_plan 01)SortPreservingMergeExec: [c_bigint@0 ASC NULLS LAST], fetch=5 02)--ProjectionExec: expr=[CAST(c@0 AS Int64) as c_bigint] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table ordered_table; @@ -1164,8 +1154,7 @@ physical_plan 01)SortPreservingMergeExec: [abs_c@0 ASC NULLS LAST], fetch=5 02)--ProjectionExec: expr=[abs(c@0) as abs_c] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true # Boolean to integer casts preserve the order. statement ok @@ -1191,8 +1180,7 @@ physical_plan 01)SortPreservingMergeExec: [c@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[CAST(inc_col@0 > desc_col@1 AS Int32) as c] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[inc_col, desc_col], output_orderings=[[inc_col@0 ASC NULLS LAST], [desc_col@1 DESC]], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[inc_col, desc_col], output_orderings=[[inc_col@0 ASC NULLS LAST], [desc_col@1 DESC]], file_type=csv, has_header=true # Union a query with the actual data and one with a constant query I @@ -1331,11 +1319,9 @@ physical_plan 02)--ProjectionExec: expr=[CASE WHEN name@0 = name1 THEN 0 WHEN name@0 = name2 THEN 0.5 END as a] 03)----UnionExec 04)------ProjectionExec: expr=[name1 as name] -05)--------YieldStreamExec child=PlaceholderRowExec -06)----------PlaceholderRowExec -07)------ProjectionExec: expr=[name2 as name] -08)--------YieldStreamExec child=PlaceholderRowExec -09)----------PlaceholderRowExec +05)--------PlaceholderRowExec +06)------ProjectionExec: expr=[name2 as name] +07)--------PlaceholderRowExec query R SELECT @@ -1373,9 +1359,7 @@ EXPLAIN SELECT c1, c2 FROM table_with_ordered_pk ORDER BY c1, c2; logical_plan 01)Sort: table_with_ordered_pk.c1 ASC NULLS LAST, table_with_ordered_pk.c2 ASC NULLS LAST 02)--TableScan: table_with_ordered_pk projection=[c1, c2] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], constraints=[PrimaryKey([0])], file_type=csv, has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], constraints=[PrimaryKey([0])], file_type=csv, has_header=true statement ok drop table table_with_ordered_pk; @@ -1404,8 +1388,7 @@ EXPLAIN SELECT c1, SUM(c2) as sum_c2 FROM table_with_ordered_not_null GROUP BY c physical_plan 01)ProjectionExec: expr=[c1@0 as c1, sum(table_with_ordered_not_null.c2)@1 as sum_c2] 02)--AggregateExec: mode=Single, gby=[c1@0 as c1], aggr=[sum(table_with_ordered_not_null.c2)], ordering_mode=Sorted -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table table_with_ordered_not_null; diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 18c5a2b8fd46..84db2d1280a7 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -121,8 +121,7 @@ logical_plan 02)--TableScan: test_table projection=[int_col, string_col] physical_plan 01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: 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], file_type=parquet +02)--DataSourceExec: 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], file_type=parquet # Add another file to the directory underlying test_table query I @@ -414,8 +413,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% statement ok @@ -463,8 +461,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -515,8 +512,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -631,8 +627,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 LIKE f% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] statement ok drop table foo diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index b31e76db8816..9bb2c5a2afbb 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -217,17 +217,14 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: val@0 != part@1 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet +04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet # If we reference only a partition column it gets evaluted during the listing phase query TT EXPLAIN select * from t_pushdown where part != 'a'; ---- logical_plan TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part != Utf8("a")] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet +physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet # And if we reference only a file column it gets pushed down query TT @@ -236,9 +233,7 @@ EXPLAIN select * from t_pushdown where val != 'c'; logical_plan 01)Filter: t_pushdown.val != Utf8("c") 02)--TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != Utf8("c")] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] +physical_plan DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] # If we have a mix of filters: # - The partition filters get evaluated during planning @@ -254,5 +249,4 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: val@0 != part@1 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] diff --git a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt index 988798908dfa..9a48c1ac003f 100644 --- a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt @@ -111,8 +111,7 @@ logical_plan 02)--TableScan: test_table projection=[int_col, bigint_col] physical_plan 01)SortPreservingMergeExec: [int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Another planning test, but project on a column with unsupported statistics # We should be able to ignore this and look at only the relevant statistics @@ -129,8 +128,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[string_col@0 as string_col] 02)--SortPreservingMergeExec: [int_col@1 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[string_col, int_col, bigint_col], output_ordering=[int_col@1 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST], file_type=parquet +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[string_col, int_col, bigint_col], output_ordering=[int_col@1 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST], file_type=parquet # Clean up & recreate but sort on descending column statement ok @@ -164,8 +162,7 @@ logical_plan 02)--TableScan: test_table projection=[descending_col, bigint_col] physical_plan 01)SortPreservingMergeExec: [descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[descending_col, bigint_col], output_ordering=[descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[descending_col, bigint_col], output_ordering=[descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Clean up & re-create with partition columns in sort order statement ok @@ -201,8 +198,7 @@ logical_plan 02)--TableScan: test_table projection=[int_col, bigint_col, partition_col] physical_plan 01)SortPreservingMergeExec: [partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, partition_col], output_ordering=[partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, partition_col], output_ordering=[partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Clean up & re-create with overlapping column in sort order # This will test the ability to sort files with overlapping statistics @@ -235,8 +231,7 @@ logical_plan 02)--TableScan: test_table projection=[int_col, bigint_col, overlapping_col] physical_plan 01)SortPreservingMergeExec: [overlapping_col@2 ASC NULLS LAST] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, overlapping_col], output_ordering=[overlapping_col@2 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, overlapping_col], output_ordering=[overlapping_col@2 ASC NULLS LAST], file_type=parquet # Clean up & re-create with constant column in sort order # This will require a sort because the # of required file groups (3) diff --git a/datafusion/sqllogictest/test_files/parquet_statistics.slt b/datafusion/sqllogictest/test_files/parquet_statistics.slt index 37ac5965d10f..c707b9f5bbd5 100644 --- a/datafusion/sqllogictest/test_files/parquet_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_statistics.slt @@ -62,9 +62,8 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] 02)--FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -04)------YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] -06), statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] +05), statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] # cleanup statement ok @@ -87,12 +86,11 @@ query TT EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -02)--FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -04)------YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] -06), statistics=[Rows=Inexact(5), Bytes=Inexact(173), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Inexact(2), Bytes=Inexact(44), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] +02)--FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(44), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(173), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] +05), statistics=[Rows=Inexact(5), Bytes=Inexact(173), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] # cleanup statement ok @@ -119,9 +117,8 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] 02)--FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -04)------YieldStreamExec child=DataSourceExec, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] -06), statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)] +05), statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] # cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 0674366dcff0..3b7dbe934cfa 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -675,15 +675,13 @@ physical_plan 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)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=true -10)----CoalesceBatchesExec: target_batch_size=8192 -11)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -12)--------CoalesceBatchesExec: target_batch_size=8192 -13)----------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 -14)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)--------------YieldStreamExec child=DataSourceExec -16)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], file_type=csv, has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], file_type=csv, 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)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], file_type=csv, has_header=true ######## # TPCH Q19 - Pull predicates to inner join (simplified) @@ -791,17 +789,13 @@ query TT explain select x from t where x IN (1,2,3) AND x IN (4,5); ---- logical_plan EmptyRelation -physical_plan -01)YieldStreamExec child=EmptyExec -02)--EmptyExec +physical_plan EmptyExec query TT explain select x from t where x NOT IN (1,2,3,4) OR x NOT IN (5,6,7,8); ---- logical_plan TableScan: t projection=[x] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select x from t where x IN (1,2,3,4,5) AND x NOT IN (1,2,3,4); @@ -812,16 +806,13 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: x@0 = 5 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: 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); ---- logical_plan EmptyRelation -physical_plan -01)YieldStreamExec child=EmptyExec -02)--EmptyExec +physical_plan EmptyExec statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/projection.slt b/datafusion/sqllogictest/test_files/projection.slt index f9c52a7720ab..0f0cbac1fa32 100644 --- a/datafusion/sqllogictest/test_files/projection.slt +++ b/datafusion/sqllogictest/test_files/projection.slt @@ -183,9 +183,7 @@ query TT explain select b from memory_table; ---- logical_plan TableScan: memory_table projection=[b] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] query I select b from memory_table; @@ -250,8 +248,7 @@ logical_plan 02)--TableScan: t projection=[column1] physical_plan 01)ProjectionExec: expr=[get_field(column1@0, c0) as t.column1[c0]] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index 244f19569970..ed948dd11439 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -42,8 +42,7 @@ physical_plan 04)------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------FilterExec: column1@0 = 2 -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query I select uc2 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3; @@ -63,8 +62,7 @@ physical_plan 05)--------ProjectionExec: expr=[__unnest_placeholder(v.column2,depth=1)@0 as __unnest_placeholder(v.column2,depth=1)] 06)----------UnnestExec 07)------------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query II select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 AND column1 = 2; @@ -85,8 +83,7 @@ physical_plan 06)----------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------FilterExec: column1@0 = 2 -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query II select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 OR column1 = 2; @@ -106,8 +103,7 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------UnnestExec 06)----------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table v; @@ -131,8 +127,7 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------UnnestExec 06)----------ProjectionExec: expr=[column1@0 as column1, column2@1 as __unnest_placeholder(d.column2)] -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -197,8 +192,7 @@ explain select * from test_filter_with_limit where value = 2 limit 1; ---- physical_plan 01)CoalescePartitionsExec: fetch=1 -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-2.parquet]]}, projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)] +02)--DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-2.parquet]]}, projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)] query II select * from test_filter_with_limit where value = 2 limit 1; @@ -235,57 +229,43 @@ LOCATION 'test_files/scratch/push_down_filter/t.parquet'; query TT explain select a from t where a = '100'; ---- -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] # The predicate should not have a column cast when the value is a valid i32 query TT explain select a from t where a != '100'; ---- -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 != 100, pruning_predicate=a_null_count@2 != row_count@3 AND (a_min@0 != 100 OR 100 != a_max@1), required_guarantees=[a not in (100)] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 != 100, pruning_predicate=a_null_count@2 != row_count@3 AND (a_min@0 != 100 OR 100 != a_max@1), required_guarantees=[a not in (100)] # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = '99999999999'; ---- -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99999999999 +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99999999999 # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = '99.99'; ---- -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99.99 +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99.99 # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = ''; ---- -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = # The predicate should not have a column cast when the operator is = or != and the literal can be round-trip casted without losing information. query TT explain select a from t where cast(a as string) = '100'; ---- -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] # The predicate should still have the column cast when the literal alters its string representation after round-trip casting (leading zero lost). query TT explain select a from t where CAST(a AS string) = '0123'; ---- -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 0123 +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 0123 statement ok diff --git a/datafusion/sqllogictest/test_files/regexp/regexp_like.slt b/datafusion/sqllogictest/test_files/regexp/regexp_like.slt index d263bcd8ac9d..8c407ea2e760 100644 --- a/datafusion/sqllogictest/test_files/regexp/regexp_like.slt +++ b/datafusion/sqllogictest/test_files/regexp/regexp_like.slt @@ -254,8 +254,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 LIKE %oo% -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # Ensure casting / coercion works for all operators # (there should be no casts to Utf8) @@ -272,8 +271,7 @@ logical_plan 02)--TableScan: dict_table projection=[column1] physical_plan 01)ProjectionExec: expr=[column1@0 LIKE %oo% as dict_table.column1 LIKE Utf8("%oo%"), column1@0 NOT LIKE %oo% as dict_table.column1 NOT LIKE Utf8("%oo%"), column1@0 ILIKE %oo% as dict_table.column1 ILIKE Utf8("%oo%"), column1@0 NOT ILIKE %oo% as dict_table.column1 NOT ILIKE Utf8("%oo%")] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table strings diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 93fc64e5fa13..05d305122b5a 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -127,8 +127,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 03)----FilterExec: c3@2 > 0 04)------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -05)--------YieldStreamExec child=StreamingTableExec -06)----------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true +05)--------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true # Start repratition on empty column test. # See https://github.com/apache/datafusion/issues/12057 diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index bf6f04c2f41f..aa0bcf377627 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -61,8 +61,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok @@ -78,8 +77,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -141,8 +139,7 @@ physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: column1@0 != 42 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..269], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..273], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:273..547], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:269..538]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..269], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..273], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:273..547], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:269..538]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # Cleanup statement ok @@ -189,8 +186,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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], file_type=csv, has_header=true +03)----DataSourceExec: 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], file_type=csv, has_header=true # Cleanup statement ok @@ -233,8 +229,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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], file_type=json +03)----DataSourceExec: 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], file_type=json # Cleanup statement ok @@ -259,9 +254,7 @@ query TT EXPLAIN SELECT * FROM arrow_table ---- logical_plan TableScan: arrow_table projection=[f0, f1, f2] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:0..461], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:461..922], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:922..1383], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:1383..1842]]}, projection=[f0, f1, f2], file_type=arrow +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:0..461], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:461..922], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:922..1383], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:1383..1842]]}, projection=[f0, f1, f2], file_type=arrow # correct content query ITB @@ -293,9 +286,7 @@ query TT EXPLAIN SELECT * FROM avro_table ---- logical_plan TableScan: avro_table projection=[f1, f2, f3] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/simple_enum.avro]]}, projection=[f1, f2, f3], file_type=avro +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/simple_enum.avro]]}, projection=[f1, f2, f3], file_type=avro # Cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt index 90ba51d13642..f583d659fd4f 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -1836,8 +1836,7 @@ logical_plan 02)--TableScan: simple_string projection=[letter] physical_plan 01)ProjectionExec: expr=[letter@0 as letter, letter@0 = A as simple_string.letter = left(Utf8("APACHE"),Int64(1))] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TB SELECT letter, letter = LEFT('APACHE', 1) FROM simple_string; @@ -1856,8 +1855,7 @@ logical_plan 02)--TableScan: simple_string projection=[letter, letter2] physical_plan 01)ProjectionExec: expr=[letter@0 as letter, letter@0 = left(letter2@1, 1) as simple_string.letter = left(simple_string.letter2,Int64(1))] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: 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 8a11b6d1da43..4e373a7ad6d0 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -424,25 +424,19 @@ query TT EXPLAIN VALUES (1, 'a', -1, 1.1),(NULL, 'b', -3, 0.5) ---- logical_plan Values: (Int64(1), Utf8("a"), Int64(-1), Float64(1.1)), (Int64(NULL), Utf8("b"), Int64(-3), Float64(0.5)) -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN VALUES ('1'::float) ---- logical_plan Values: (Float32(1) AS Utf8("1")) -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN VALUES (('1'||'2')::int unsigned) ---- logical_plan Values: (UInt32(12) AS Utf8("1") || Utf8("2")) -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] # all where empty @@ -941,8 +935,7 @@ logical_plan 02)--TableScan: select_between_data projection=[c1] physical_plan 01)ProjectionExec: expr=[c1@0 >= 2 AND c1@0 <= 3 as select_between_data.c1 BETWEEN Int64(2) AND Int64(3)] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # TODO: query_get_indexed_field @@ -1205,9 +1198,7 @@ EXPLAIN SELECT a FROM annotated_data_finite2 logical_plan 01)Sort: annotated_data_finite2.a ASC NULLS LAST, fetch=5 02)--TableScan: annotated_data_finite2 projection=[a] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], limit=5, output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], limit=5, output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true query I SELECT a FROM annotated_data_finite2 @@ -1417,8 +1408,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: 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], file_type=csv, has_header=true +04)------DataSourceExec: 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], file_type=csv, has_header=true # since query below doesn't computation # inside projection expr, increasing partitions @@ -1435,8 +1425,7 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, b@1 as b, 2 as Int64(2)] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: 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], file_type=csv, has_header=true +02)--DataSourceExec: 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], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a @@ -1457,8 +1446,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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], file_type=csv, has_header=true +05)--------DataSourceExec: 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], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1479,8 +1467,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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], file_type=csv, has_header=true +05)--------DataSourceExec: 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], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1501,8 +1488,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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], file_type=csv, has_header=true +05)--------DataSourceExec: 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], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1523,8 +1509,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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], file_type=csv, has_header=true +05)--------DataSourceExec: 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], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is when filter contains or @@ -1596,8 +1581,7 @@ physical_plan 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)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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], file_type=csv, has_header=true +05)--------DataSourceExec: 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], file_type=csv, has_header=true statement ok @@ -1625,8 +1609,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as t.x / Int64(2), __common_expr_1@0 + 1 as t.x / Int64(2) + Int64(1)] 02)--ProjectionExec: expr=[x@0 / 2 as __common_expr_1] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT x/2, x/2+1 FROM t; @@ -1644,8 +1627,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as abs(t.x), __common_expr_1@0 + abs(y@1) as abs(t.x) + abs(t.y)] 02)--ProjectionExec: expr=[abs(x@0) as __common_expr_1, y@1 as y] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT abs(x), abs(x) + abs(y) FROM t; @@ -1683,8 +1665,7 @@ logical_plan 02)--TableScan: t projection=[x, y] physical_plan 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)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: 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; @@ -1696,8 +1677,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 AND 1 / CAST(y@2 AS Int64) < 1 as t.y > Int64(0) AND Int64(1) / t.y < Int64(1), x@1 > 0 AND __common_expr_1@0 AND 1 / CAST(y@2 AS Int64) < 1 / CAST(x@1 AS Int64) as t.x > Int64(0) AND t.y > Int64(0) AND Int64(1) / t.y < Int64(1) / t.x] 02)--ProjectionExec: expr=[y@1 > 0 as __common_expr_1, x@0 as x, y@1 as y] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: 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; @@ -1709,8 +1689,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 OR 1 / CAST(y@2 AS Int64) < 1 as t.y = Int64(0) OR Int64(1) / t.y < Int64(1), x@1 = 0 OR __common_expr_1@0 OR 1 / CAST(y@2 AS Int64) < 1 / CAST(x@1 AS Int64) as t.x = Int64(0) OR t.y = Int64(0) OR Int64(1) / t.y < Int64(1) / t.x] 02)--ProjectionExec: expr=[y@1 = 0 as __common_expr_1, x@0 as x, y@1 as y] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # due to the reason describe in https://github.com/apache/datafusion/issues/8927, # the following queries will fail diff --git a/datafusion/sqllogictest/test_files/simplify_expr.slt b/datafusion/sqllogictest/test_files/simplify_expr.slt index 056b4676d40e..075ccafcfd2e 100644 --- a/datafusion/sqllogictest/test_files/simplify_expr.slt +++ b/datafusion/sqllogictest/test_files/simplify_expr.slt @@ -28,8 +28,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: a@0 = 3 -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # test regex exprs query TT @@ -41,8 +40,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: b@0 IS NOT NULL -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select b from t where b !~ '.*' @@ -53,8 +51,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: b@0 = -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query T select b from t where b ~ '.*' @@ -75,8 +72,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: a@0 IS NOT NULL OR NULL -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/struct.slt b/datafusion/sqllogictest/test_files/struct.slt index ef90c3ff8daa..46e15a4d6d10 100644 --- a/datafusion/sqllogictest/test_files/struct.slt +++ b/datafusion/sqllogictest/test_files/struct.slt @@ -124,8 +124,7 @@ logical_plan 02)--TableScan: values projection=[a, b, c] physical_plan 01)ProjectionExec: expr=[struct(a@0, b@1, c@2) as struct(values.a,values.b,values.c)] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # error on 0 arguments query error diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 044fd32fc1ee..65152b621e42 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -213,8 +213,7 @@ physical_plan 11)--------------------YieldStreamExec child=DataSourceExec 12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] 13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------YieldStreamExec child=DataSourceExec -15)----------DataSourceExec: partitions=1, partition_sizes=[1] +14)--------DataSourceExec: partitions=1, partition_sizes=[1] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -250,8 +249,7 @@ physical_plan 11)--------------------YieldStreamExec child=DataSourceExec 12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] 13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------YieldStreamExec child=DataSourceExec -15)----------DataSourceExec: partitions=1, partition_sizes=[1] +14)--------DataSourceExec: partitions=1, partition_sizes=[1] 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 @@ -287,8 +285,7 @@ physical_plan 11)--------------------YieldStreamExec child=DataSourceExec 12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] 13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------YieldStreamExec child=DataSourceExec -15)----------DataSourceExec: partitions=1, partition_sizes=[1] +14)--------DataSourceExec: partitions=1, partition_sizes=[1] 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 @@ -327,8 +324,7 @@ physical_plan 13)------------------------YieldStreamExec child=DataSourceExec 14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] 15)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)--------YieldStreamExec child=DataSourceExec -17)----------DataSourceExec: partitions=1, partition_sizes=[1] +16)--------DataSourceExec: partitions=1, partition_sizes=[1] 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 @@ -781,11 +777,9 @@ physical_plan 01)ProjectionExec: expr=[count(*)@0 as b, count(Int64(1))@1 as count(Int64(1))] 02)--NestedLoopJoinExec: join_type=Left 03)----ProjectionExec: expr=[4 as count(*)] -04)------YieldStreamExec child=PlaceholderRowExec -05)--------PlaceholderRowExec -06)----ProjectionExec: expr=[4 as count(Int64(1))] -07)------YieldStreamExec child=PlaceholderRowExec -08)--------PlaceholderRowExec +04)------PlaceholderRowExec +05)----ProjectionExec: expr=[4 as count(Int64(1))] +06)------PlaceholderRowExec statement ok set datafusion.explain.logical_plan_only = true; @@ -1203,11 +1197,9 @@ physical_plan 02)--FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/subquery_sort.slt b/datafusion/sqllogictest/test_files/subquery_sort.slt index 4e8a1c9318c0..4f873d259df2 100644 --- a/datafusion/sqllogictest/test_files/subquery_sort.slt +++ b/datafusion/sqllogictest/test_files/subquery_sort.slt @@ -43,17 +43,13 @@ EXPLAIN SELECT c1 FROM (SELECT c1 FROM sink_table ORDER BY c2) AS ttt logical_plan 01)SubqueryAlias: ttt 02)--TableScan: sink_table projection=[c1] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query TT EXPLAIN SELECT c1 FROM (SELECT c1 FROM sink_table ORDER BY c2) ---- logical_plan TableScan: sink_table projection=[c1] -physical_plan -01)YieldStreamExec child=DataSourceExec -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true # Do not remove ordering when it's with limit diff --git a/datafusion/sqllogictest/test_files/table_functions.slt b/datafusion/sqllogictest/test_files/table_functions.slt index 16dc8f2cea81..7d318c50bacf 100644 --- a/datafusion/sqllogictest/test_files/table_functions.slt +++ b/datafusion/sqllogictest/test_files/table_functions.slt @@ -154,9 +154,7 @@ query TT EXPLAIN SELECT * FROM generate_series(1, 5) ---- logical_plan TableScan: tmp_table projection=[value] -physical_plan -01)YieldStreamExec child=LazyMemoryExec -02)--LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=5, batch_size=8192] +physical_plan LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=5, batch_size=8192] # # Test generate_series with invalid arguments @@ -273,9 +271,7 @@ query TT EXPLAIN SELECT * FROM range(1, 5) ---- logical_plan TableScan: tmp_table projection=[value] -physical_plan -01)YieldStreamExec child=LazyMemoryExec -02)--LazyMemoryExec: partitions=1, batch_generators=[range: start=1, end=5, batch_size=8192] +physical_plan LazyMemoryExec: partitions=1, batch_generators=[range: start=1, end=5, batch_size=8192] # # Test range with invalid arguments diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 6df5b5d234fc..9cdfef802b87 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -273,13 +273,10 @@ logical_plan 05)----TableScan: t2 projection=[name] physical_plan 01)UnionExec -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)--YieldStreamExec child=DataSourceExec +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--DataSourceExec: partitions=1, partition_sizes=[1] +04)--ProjectionExec: expr=[name@0 || _new as name] 05)----DataSourceExec: partitions=1, partition_sizes=[1] -06)--ProjectionExec: expr=[name@0 || _new as name] -07)----YieldStreamExec child=DataSourceExec -08)------DataSourceExec: partitions=1, partition_sizes=[1] # Make sure to choose a small batch size to introduce parallelism to the plan. statement ok @@ -325,23 +322,21 @@ physical_plan 11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] 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)----------YieldStreamExec child=DataSourceExec -15)------------DataSourceExec: partitions=1, partition_sizes=[1] -16)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] -17)----CoalesceBatchesExec: target_batch_size=2 -18)------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] -19)--------CoalescePartitionsExec -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)----------------------YieldStreamExec child=DataSourceExec -27)------------------------DataSourceExec: partitions=1, partition_sizes=[1] -28)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)----------YieldStreamExec child=DataSourceExec -30)------------DataSourceExec: partitions=1, partition_sizes=[1] +14)----------DataSourceExec: 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=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] +18)--------CoalescePartitionsExec +19)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +20)------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +21)--------------CoalesceBatchesExec: target_batch_size=2 +22)----------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +23)------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +24)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +25)----------------------YieldStreamExec child=DataSourceExec +26)------------------------DataSourceExec: partitions=1, partition_sizes=[1] +27)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)----------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort @@ -398,21 +393,19 @@ physical_plan 10)------------------YieldStreamExec child=DataSourceExec 11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] 12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------YieldStreamExec child=DataSourceExec -14)----------DataSourceExec: partitions=1, partition_sizes=[1] -15)--CoalesceBatchesExec: target_batch_size=2 -16)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)] -17)------CoalescePartitionsExec -18)--------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] -19)----------CoalesceBatchesExec: target_batch_size=2 -20)------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -21)--------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -22)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -23)------------------YieldStreamExec child=DataSourceExec -24)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -25)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -26)--------YieldStreamExec child=DataSourceExec -27)----------DataSourceExec: partitions=1, partition_sizes=[1] +13)--------DataSourceExec: partitions=1, partition_sizes=[1] +14)--CoalesceBatchesExec: target_batch_size=2 +15)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)] +16)------CoalescePartitionsExec +17)--------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +18)----------CoalesceBatchesExec: target_batch_size=2 +19)------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +20)--------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +21)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +22)------------------YieldStreamExec child=DataSourceExec +23)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +24)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +25)--------DataSourceExec: partitions=1, partition_sizes=[1] # union_upcast_types query TT @@ -624,11 +617,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST] 02)--UnionExec -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true -05)----ProjectionExec: expr=[c1a@0 as c1] -06)------YieldStreamExec child=DataSourceExec -07)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1a], output_ordering=[c1a@0 ASC NULLS LAST], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +04)----ProjectionExec: expr=[c1a@0 as c1] +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1a], output_ordering=[c1a@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table t1 @@ -653,14 +644,11 @@ physical_plan 01)UnionExec 02)--ProjectionExec: expr=[Int64(1)@0 as a] 03)----AggregateExec: mode=SinglePartitioned, gby=[1 as Int64(1)], aggr=[], ordering_mode=Sorted -04)------YieldStreamExec child=PlaceholderRowExec -05)--------PlaceholderRowExec -06)--ProjectionExec: expr=[2 as a] -07)----YieldStreamExec child=PlaceholderRowExec -08)------PlaceholderRowExec -09)--ProjectionExec: expr=[3 as a] -10)----YieldStreamExec child=PlaceholderRowExec -11)------PlaceholderRowExec +04)------PlaceholderRowExec +05)--ProjectionExec: expr=[2 as a] +06)----PlaceholderRowExec +07)--ProjectionExec: expr=[3 as a] +08)----PlaceholderRowExec # test UNION ALL aliases correctly with aliased subquery query TT @@ -867,13 +855,11 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------FilterExec: c1@0 = a 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: 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], file_type=csv, has_header=true -08)----CoalesceBatchesExec: target_batch_size=2 -09)------FilterExec: c1@0 = a -10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)----------YieldStreamExec child=DataSourceExec -12)------------DataSourceExec: 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], file_type=csv, has_header=true +06)----------DataSourceExec: 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], file_type=csv, has_header=true +07)----CoalesceBatchesExec: target_batch_size=2 +08)------FilterExec: c1@0 = a +09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)----------DataSourceExec: 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], file_type=csv, has_header=true # Clean up after the test statement ok @@ -975,10 +961,8 @@ physical_plan 01)UnionExec 02)--ProjectionExec: expr=[CAST(y@0 AS Int64) as y] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)--YieldStreamExec child=DataSourceExec -07)----DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)--DataSourceExec: partitions=1, partition_sizes=[1] statement count 0 drop view v1; diff --git a/datafusion/sqllogictest/test_files/unnest.slt b/datafusion/sqllogictest/test_files/unnest.slt index 2e8a10864a55..92e6f9995ae3 100644 --- a/datafusion/sqllogictest/test_files/unnest.slt +++ b/datafusion/sqllogictest/test_files/unnest.slt @@ -605,8 +605,7 @@ physical_plan 03)----ProjectionExec: expr=[__unnest_placeholder(recursive_unnest_table.column3,depth=1)@0 as __unnest_placeholder(UNNEST(recursive_unnest_table.column3)), column3@1 as column3] 04)------UnnestExec 05)--------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] ## unnest->field_access->unnest->unnest query I? @@ -664,8 +663,7 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------UnnestExec 06)----------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 2c5af1259813..9bd7188e878d 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2878,8 +2878,7 @@ physical_plan 03)----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: false, 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] 04)------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: false, 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] 05)--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] -06)----------YieldStreamExec child=StreamingTableExec -07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII SELECT @@ -2924,8 +2923,7 @@ physical_plan 03)----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: false, 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] 04)------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: false, 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] 05)--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] -06)----------YieldStreamExec child=StreamingTableExec -07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII @@ -3029,8 +3027,7 @@ physical_plan 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 __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -10)------------------YieldStreamExec child=StreamingTableExec -11)--------------------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] +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, @@ -3167,8 +3164,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=4096, fetch=5 03)----FilterExec: row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 < 50 04)------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] -05)--------YieldStreamExec child=StreamingTableExec -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] +05)--------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] # Top level sort is pushed down through BoundedWindowAggExec as its SUM result does already satisfy the required # global order. The existing sort is for the second-term lexicographical ordering requirement, which is being @@ -3281,8 +3277,7 @@ physical_plan 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 __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -08)--------------YieldStreamExec child=StreamingTableExec -09)----------------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] +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; @@ -3322,8 +3317,7 @@ physical_plan 14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -17)--------------------------------YieldStreamExec child=StreamingTableExec -18)----------------------------------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] +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 @@ -3478,8 +3472,7 @@ physical_plan 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)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: 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]], file_type=csv, has_header=true +04)------DataSourceExec: 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]], file_type=csv, has_header=true # Since column b is constant after filter b=0, # window requirement b ASC, d ASC can be satisfied @@ -3535,8 +3528,7 @@ physical_plan 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)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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]], file_type=csv, has_header=true +05)--------DataSourceExec: 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]], file_type=csv, has_header=true query TT EXPLAIN SELECT MAX(c) OVER(PARTITION BY d ORDER BY c ASC) as max_c @@ -3555,8 +3547,7 @@ physical_plan 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)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c ORDER BY a ASC) @@ -3569,8 +3560,7 @@ logical_plan physical_plan 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)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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]], file_type=csv, has_header=true +03)----DataSourceExec: 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]], file_type=csv, has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) @@ -3583,8 +3573,7 @@ logical_plan physical_plan 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)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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]], file_type=csv, has_header=true +03)----DataSourceExec: 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]], file_type=csv, has_header=true query I SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) @@ -3681,8 +3670,7 @@ physical_plan 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)------------YieldStreamExec child=StreamingTableExec -08)--------------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]] +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 @@ -4181,8 +4169,7 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=4096 07)------------FilterExec: a@0 = 1 -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4205,8 +4192,7 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=4096 07)------------FilterExec: a@0 = 1 -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT From 021fb9219ccc8cf2c727853b21b3e973250b64f2 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 2 Jun 2025 21:38:23 +0800 Subject: [PATCH 20/74] fix test --- datafusion/core/src/physical_planner.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 454911afcde0..fbb4250fc4df 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -3050,10 +3050,8 @@ mod tests { digraph { 1[shape=box label="ProjectionExec: expr=[id@0 + 2 as employee.id + Int32(2)]", tooltip=""] - 2[shape=box label="YieldStreamExec child=EmptyExec", tooltip=""] + 2[shape=box label="EmptyExec", tooltip=""] 1 -> 2 [arrowhead=none, arrowtail=normal, dir=back] - 3[shape=box label="EmptyExec", tooltip=""] - 2 -> 3 [arrowhead=none, arrowtail=normal, dir=back] } // End DataFusion GraphViz Plan "#; From 5027087eacfaed1e0a4d37322e243bf614f53825 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 2 Jun 2025 21:50:08 +0800 Subject: [PATCH 21/74] polish code --- datafusion/core/src/execution/session_state.rs | 2 +- datafusion/physical-plan/src/yield_stream.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 0e6408b531fe..8aa812cc5258 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -2177,7 +2177,7 @@ mod tests { let displayable = DisplayableExecutionPlan::new(got.as_ref()); assert_eq!( displayable.indent(false).to_string(), - "ProjectionExec: expr=[0 as count(*)]\n YieldStreamExec\n PlaceholderRowExec\n" + "ProjectionExec: expr=[0 as count(*)]\n PlaceholderRowExec\n" ); Ok(()) diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index 61cdf5f58aab..718907fc0b5b 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -122,7 +122,7 @@ impl DisplayAs for YieldStreamExec { impl ExecutionPlan for YieldStreamExec { fn name(&self) -> &str { - "yield_stream_exec" + "YieldStreamExec" } fn as_any(&self) -> &dyn Any { From 8509d0a879b7b833a10189f6b8a5d7feeae40372 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 2 Jun 2025 23:07:45 +0800 Subject: [PATCH 22/74] fix testing and address suggestions --- .../core/tests/dataframe/.mod.rs.pending-snap | 787 ++++++++++++++++++ datafusion/core/tests/dataframe/mod.rs | 89 +- datafusion/core/tests/memory_limit/mod.rs | 19 +- .../enforce_distribution.rs | 3 +- datafusion/core/tests/sql/explain_analyze.rs | 3 +- datafusion/physical-plan/src/yield_stream.rs | 30 +- 6 files changed, 880 insertions(+), 51 deletions(-) create mode 100644 datafusion/core/tests/dataframe/.mod.rs.pending-snap diff --git a/datafusion/core/tests/dataframe/.mod.rs.pending-snap b/datafusion/core/tests/dataframe/.mod.rs.pending-snap new file mode 100644 index 000000000000..36509e904b10 --- /dev/null +++ b/datafusion/core/tests/dataframe/.mod.rs.pending-snap @@ -0,0 +1,787 @@ +{"run_id":"1748873147-503907000","line":571,"new":{"module_name":"core_integration__dataframe","snapshot_name":"aggregate_with_pk","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":571,"expression":"physical_plan_to_string(&df).await"},"snapshot":"AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]\n YieldStreamExec child=DataSourceExec\n DataSourceExec: partitions=1, partition_sizes=[1]"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]\n DataSourceExec: partitions=1, partition_sizes=[1]"}} +{"run_id":"1748873175-977737000","line":571,"new":null,"old":null} +{"run_id":"1748873175-977737000","line":582,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4557,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5490,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2022,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":511,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1920,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5509,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5548,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5561,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5417,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2029,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5435,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":480,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":448,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5450,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3221,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3203,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3266,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1197,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1243,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1170,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3506,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1219,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1224,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3517,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3598,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3239,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3551,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3610,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":342,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2310,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3302,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3564,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3167,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3185,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5334,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":874,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3080,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5352,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":571,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3127,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":582,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5367,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":613,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5647,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":663,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5658,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":715,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5720,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":678,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":628,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":728,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5725,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5755,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5767,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5696,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5783,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4878,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":756,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1389,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1405,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":839,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2716,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4973,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2635,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5042,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4985,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5003,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5055,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5075,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5011,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2755,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5084,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2857,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4909,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4922,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4942,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2883,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4952,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2681,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1042,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1129,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":803,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5943,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5971,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1890,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5988,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1061,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":6066,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3337,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4508,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1096,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4474,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1013,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4053,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4153,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3810,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5583,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4181,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4074,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3832,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3851,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3630,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3876,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3646,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3930,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5615,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4205,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3898,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3952,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3760,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4001,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4421,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4022,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4348,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3672,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3788,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4379,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4249,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4282,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3700,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1986,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1710,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3429,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":4293,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":3380,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1532,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1447,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1830,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2479,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2422,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1674,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":931,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1574,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1587,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1470,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1603,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1846,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2790,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2790,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748873250-690022000","line":5148,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2364,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2538,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1745,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1758,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2575,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_sort","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2575,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.b, count(*) |\n| | Sort: count(Int64(1)) AS count(*) AS count(*) ASC NULLS LAST |\n| | Projection: t1.b, count(Int64(1)) AS count(*), count(Int64(1)) |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | ProjectionExec: expr=[b@0 as b, count(*)@1 as count(*)] |\n| | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] |\n| | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] |\n| | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.b, count(*) |\n| | Sort: count(Int64(1)) AS count(*) AS count(*) ASC NULLS LAST |\n| | Projection: t1.b, count(Int64(1)) AS count(*), count(Int64(1)) |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | ProjectionExec: expr=[b@0 as b, count(*)@1 as count(*)] |\n| | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] |\n| | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] |\n| | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748873250-690022000","line":1774,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":5170,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1493,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":2912,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_where_scalar_subquery","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2912,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.a, t1.b |\n| | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) |\n| | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true |\n| | Left Join: t1.a = __scalar_sq_1.a |\n| | TableScan: t1 projection=[a, b] |\n| | SubqueryAlias: __scalar_sq_1 |\n| | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true |\n| | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] |\n| | TableScan: t2 projection=[a] |\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\n| | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.a, t1.b |\n| | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) |\n| | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true |\n| | Left Join: t1.a = __scalar_sq_1.a |\n| | TableScan: t1 projection=[a, b] |\n| | SubqueryAlias: __scalar_sq_1 |\n| | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true |\n| | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] |\n| | TableScan: t2 projection=[a] |\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\n| | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748873250-690022000","line":1618,"new":null,"old":null} +{"run_id":"1748873250-690022000","line":1789,"new":null,"old":null} +{"run_id":"1748873447-22072000","line":2575,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_sort","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2575,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.b, count(*) |\n| | Sort: count(Int64(1)) AS count(*) AS count(*) ASC NULLS LAST |\n| | Projection: t1.b, count(Int64(1)) AS count(*), count(Int64(1)) |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | ProjectionExec: expr=[b@0 as b, count(*)@1 as count(*)] |\n| | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] |\n| | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] |\n| | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.b, count(*) |\n| | Sort: count(Int64(1)) AS count(*) AS count(*) ASC NULLS LAST |\n| | Projection: t1.b, count(Int64(1)) AS count(*), count(Int64(1)) |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | ProjectionExec: expr=[b@0 as b, count(*)@1 as count(*)] |\n| | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] |\n| | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] |\n| | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748873613-663955000","line":2575,"new":null,"old":null} +{"run_id":"1748873613-663955000","line":2602,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_sort-2","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2602,"expression":"pretty_format_batches(&df_results).unwrap()"},"snapshot":"+---------------+--------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+--------------------------------------------------------------------------------+\n| logical_plan | Sort: count(*) ASC NULLS LAST |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] |\n| | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+--------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+--------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+--------------------------------------------------------------------------------+\n| logical_plan | Sort: count(*) ASC NULLS LAST |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] |\n| | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+--------------------------------------------------------------------------------+"}} +{"run_id":"1748873676-404273000","line":2575,"new":null,"old":null} +{"run_id":"1748873676-404273000","line":2602,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2022,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5493,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4560,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":511,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1920,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5512,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5551,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5564,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2029,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5420,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5438,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":480,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5453,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":448,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3206,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3224,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3269,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1243,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1197,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1170,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3509,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1219,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1224,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3601,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3554,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3613,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3520,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3242,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2310,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":342,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3567,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3305,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3188,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3170,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":874,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5337,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3083,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5355,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5370,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":571,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":613,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3130,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":582,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5650,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":628,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":715,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5661,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":663,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":728,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5723,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5758,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5699,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5728,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":678,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5770,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5786,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":756,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1389,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4881,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1405,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":839,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2719,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2758,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4976,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":803,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5045,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2638,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2575,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2602,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5058,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4988,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5078,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5006,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2860,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4912,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5087,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5014,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4925,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2886,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1129,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4945,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4955,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1042,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2684,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5974,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5946,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1890,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5991,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4477,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":6069,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4511,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3340,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1061,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1096,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1013,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4056,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4156,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5618,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5586,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4077,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3879,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3633,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3813,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4184,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3763,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3901,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3933,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3835,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3649,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3791,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4004,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3955,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3854,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4208,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4252,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4424,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4025,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4351,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4285,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3675,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4382,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":4296,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3703,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1710,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1986,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1532,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3432,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2422,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1447,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2479,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":3383,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":931,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1574,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1587,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1674,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1603,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1830,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5151,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2364,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1470,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2538,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":5173,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1745,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1758,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1846,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1774,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748873683-425586000","line":2915,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_where_scalar_subquery","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2915,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.a, t1.b |\n| | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) |\n| | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true |\n| | Left Join: t1.a = __scalar_sq_1.a |\n| | TableScan: t1 projection=[a, b] |\n| | SubqueryAlias: __scalar_sq_1 |\n| | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true |\n| | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] |\n| | TableScan: t2 projection=[a] |\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\n| | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.a, t1.b |\n| | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) |\n| | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true |\n| | Left Join: t1.a = __scalar_sq_1.a |\n| | TableScan: t1 projection=[a, b] |\n| | SubqueryAlias: __scalar_sq_1 |\n| | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true |\n| | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] |\n| | TableScan: t2 projection=[a] |\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\n| | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748873683-425586000","line":1493,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1618,"new":null,"old":null} +{"run_id":"1748873683-425586000","line":1789,"new":null,"old":null} +{"run_id":"1748873859-597000000","line":2915,"new":null,"old":null} +{"run_id":"1748873859-597000000","line":2975,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5499,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2022,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4566,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":511,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5518,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5557,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5570,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1920,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5426,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2029,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5444,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5459,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":480,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":448,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3230,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3212,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1170,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3275,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1197,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1243,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1219,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1224,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3515,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3607,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3560,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3526,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3619,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2310,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3248,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3573,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":342,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3194,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3176,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3311,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":874,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5343,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3089,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5361,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3136,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":571,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5376,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":663,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":582,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":613,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":715,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5656,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":678,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":628,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5667,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5729,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":728,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5764,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5734,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4887,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5776,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5792,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5705,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1389,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1405,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":756,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":839,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2719,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2638,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4982,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2860,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4994,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5012,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5020,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2886,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2758,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5051,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4918,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5064,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2575,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2684,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2602,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4931,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4951,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5084,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2915,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1129,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4961,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5093,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1042,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":803,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5980,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5952,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5997,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2975,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":6075,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4517,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4483,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3346,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1013,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1061,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4062,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5592,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4162,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3819,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1096,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3841,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4083,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3639,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3860,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3885,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3655,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5624,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4190,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3939,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3907,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3769,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3961,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4010,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4031,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3797,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4357,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4430,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3681,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4258,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4388,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4214,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4291,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3709,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":4302,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1986,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1890,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1710,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1532,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1447,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3389,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2422,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":3438,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":931,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2479,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1830,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1574,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1587,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1674,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1470,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2538,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1603,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2364,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5157,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":5179,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1846,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1745,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1758,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1493,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1774,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748873866-668387000","line":1618,"new":null,"old":null} +{"run_id":"1748873866-668387000","line":1789,"new":null,"old":null} +{"run_id":"1748874092-266512000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748874158-473190000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748874308-762369000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748874553-675455000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748874806-77292000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748874842-394238000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748874912-217000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748874955-173216000","line":5503,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1920,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2022,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4570,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":511,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5522,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5561,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5574,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2029,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5430,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5448,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":480,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5463,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":448,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3234,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3216,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3279,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1243,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1197,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1170,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1219,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1224,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3519,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3611,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3564,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3530,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2310,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3252,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3623,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3577,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":342,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3198,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3180,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3315,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5347,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":874,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5365,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5380,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3093,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3140,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":571,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":582,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5660,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":663,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":613,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5671,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":715,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":678,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5733,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":628,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5738,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":728,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5709,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5768,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4891,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5780,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5796,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":756,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1389,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1405,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":839,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2719,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2864,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4986,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2638,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4998,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5016,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5024,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2890,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5055,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4922,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2575,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2602,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4935,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4955,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5068,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5088,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4965,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2758,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2919,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5097,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2684,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1129,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1042,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5984,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5956,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2979,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":6001,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":6079,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1890,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4487,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3350,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4521,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1061,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1013,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5596,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4166,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4066,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3823,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3845,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1096,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3643,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3864,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4194,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4087,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3659,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3943,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5628,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3965,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4218,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3889,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4014,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3773,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3911,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":803,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4361,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3801,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4035,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4392,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4434,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4262,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3685,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4295,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1986,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3713,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":4306,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1710,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1532,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1447,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3442,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2422,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1830,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":3393,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2479,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":931,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1674,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1470,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2364,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5161,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1574,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1846,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1587,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2538,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":5183,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1603,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1745,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1758,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1493,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1774,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748874955-173216000","line":1618,"new":null,"old":null} +{"run_id":"1748874955-173216000","line":1789,"new":null,"old":null} +{"run_id":"1748875050-86297000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748875102-610079000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748875163-578276000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748875364-204310000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748875457-236822000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748875515-685334000","line":2795,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2795,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748875568-55246000","line":2795,"new":null,"old":null} +{"run_id":"1748875568-55246000","line":2831,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window-2","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2831,"expression":"pretty_format_batches(&df_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748875609-922318000","line":2793,"new":null,"old":null} +{"run_id":"1748875609-922318000","line":2832,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window-2","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2832,"expression":"pretty_format_batches(&df_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} +{"run_id":"1748875647-407491000","line":2793,"new":null,"old":null} +{"run_id":"1748875647-407491000","line":2832,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5499,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5518,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5557,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5570,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":511,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2022,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1920,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4566,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5426,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2029,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5444,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5459,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":480,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":448,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3230,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3212,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3275,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1243,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1197,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1219,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1224,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3515,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1170,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3526,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3607,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3560,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2310,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3619,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3248,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3573,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3311,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":342,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3176,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3194,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3136,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":874,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":571,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5343,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":582,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3089,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":613,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5361,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":628,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5729,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5734,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5376,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":715,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5656,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5667,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":663,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":728,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":678,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4887,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5705,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5764,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":756,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5776,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5792,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1389,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1405,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2719,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2575,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2602,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":839,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2862,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2758,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2638,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2888,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4982,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4994,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5012,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4918,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5051,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5020,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2793,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4931,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4951,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5064,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5084,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4961,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2828,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2684,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5093,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1129,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1042,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5952,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2917,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5980,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5997,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3346,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":6075,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1096,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1061,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2976,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4483,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4517,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1013,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4062,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5592,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4083,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4162,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5624,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1890,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3639,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4190,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3819,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3769,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3885,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3841,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3907,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3860,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3655,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3797,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4214,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3939,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4010,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4357,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3961,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4031,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3681,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4388,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4430,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4258,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4291,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3709,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":803,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":4302,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1710,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1986,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1447,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2422,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1532,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2479,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3389,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1674,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1830,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1574,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1587,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":931,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1603,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":3438,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1470,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2364,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":2538,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5157,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1745,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1758,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1846,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":5179,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1774,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1493,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1618,"new":null,"old":null} +{"run_id":"1748876842-95102000","line":1789,"new":null,"old":null} diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 089ff8808134..37fd2edafe50 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -572,7 +572,8 @@ async fn test_aggregate_with_pk() -> Result<()> { physical_plan_to_string(&df).await, @r###" AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[] - DataSourceExec: partitions=1, partition_sizes=[1] + YieldStreamExec child=DataSourceExec + DataSourceExec: partitions=1, partition_sizes=[1] "### ); @@ -615,7 +616,8 @@ async fn test_aggregate_with_pk2() -> Result<()> { CoalesceBatchesExec: target_batch_size=8192 FilterExec: id@0 = 1 AND name@1 = a AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[] - DataSourceExec: partitions=1, partition_sizes=[1] + YieldStreamExec child=DataSourceExec + DataSourceExec: partitions=1, partition_sizes=[1] "### ); @@ -664,7 +666,8 @@ async fn test_aggregate_with_pk3() -> Result<()> { CoalesceBatchesExec: target_batch_size=8192 FilterExec: id@0 = 1 AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[] - DataSourceExec: partitions=1, partition_sizes=[1] + YieldStreamExec child=DataSourceExec + DataSourceExec: partitions=1, partition_sizes=[1] "### ); @@ -715,7 +718,8 @@ async fn test_aggregate_with_pk4() -> Result<()> { CoalesceBatchesExec: target_batch_size=8192 FilterExec: id@0 = 1 AggregateExec: mode=Single, gby=[id@0 as id], aggr=[] - DataSourceExec: partitions=1, partition_sizes=[1] + YieldStreamExec child=DataSourceExec + DataSourceExec: partitions=1, partition_sizes=[1] "### ); @@ -2588,7 +2592,8 @@ async fn test_count_wildcard_on_sort() -> Result<()> { | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | YieldStreamExec child=DataSourceExec | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+------------------------------------------------------------------------------------------------------------+ "### @@ -2610,11 +2615,13 @@ async fn test_count_wildcard_on_sort() -> Result<()> { | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | YieldStreamExec child=DataSourceExec | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+--------------------------------------------------------------------------------+ "### ); + Ok(()) } @@ -2784,20 +2791,21 @@ async fn test_count_wildcard_on_window() -> Result<()> { .await?; assert_snapshot!( - pretty_format_batches(&sql_results).unwrap(), - @r###" - +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING | - | | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] | - | | TableScan: t1 projection=[a] | - | physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] | - | | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] | - | | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + pretty_format_batches(&sql_results).unwrap(), + @r###" ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING | +| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] | +| | TableScan: t1 projection=[a] | +| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] | +| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] | +| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] | +| | YieldStreamExec child=DataSourceExec | +| | DataSourceExec: partitions=1, partition_sizes=[1] | +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ "### ); @@ -2818,20 +2826,21 @@ async fn test_count_wildcard_on_window() -> Result<()> { .await?; assert_snapshot!( - pretty_format_batches(&df_results).unwrap(), - @r###" - +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING | - | | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] | - | | TableScan: t1 projection=[a] | - | physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] | - | | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] | - | | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + pretty_format_batches(&df_results).unwrap(), + @r###" ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING | +| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] | +| | TableScan: t1 projection=[a] | +| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] | +| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] | +| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] | +| | YieldStreamExec child=DataSourceExec | +| | DataSourceExec: partitions=1, partition_sizes=[1] | +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ "### ); @@ -2924,14 +2933,16 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | | | CoalesceBatchesExec: target_batch_size=8192 | | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | YieldStreamExec child=DataSourceExec | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | | | CoalesceBatchesExec: target_batch_size=8192 | | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | YieldStreamExec child=DataSourceExec | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+---------------------------------------------------------------------------------------------------------------------------+ " @@ -2981,14 +2992,16 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | | | CoalesceBatchesExec: target_batch_size=8192 | | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | YieldStreamExec child=DataSourceExec | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | | | CoalesceBatchesExec: target_batch_size=8192 | | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | YieldStreamExec child=DataSourceExec | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+---------------------------------------------------------------------------------------------------------------------------+ " diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 7695cc0969d8..fbee8684f3f2 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -294,15 +294,16 @@ async fn sort_spill_reservation() { // also merge, so we can ensure the sort could finish // given enough merging memory &[ - "+---------------+-------------------------------------------------------------------------------------------------------------+", - "| plan_type | plan |", - "+---------------+-------------------------------------------------------------------------------------------------------------+", - "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", - "| | TableScan: t projection=[a, b] |", - "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] |", - "| | DataSourceExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", - "| | |", - "+---------------+-------------------------------------------------------------------------------------------------------------+", + "+---------------+---------------------------------------------------------------------------------------------------------------+", + "| plan_type | plan |", + "+---------------+---------------------------------------------------------------------------------------------------------------+", + "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", + "| | TableScan: t projection=[a, b] |", + "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] |", + "| | YieldStreamExec child=DataSourceExec |", + "| | DataSourceExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", + "| | |", + "+---------------+---------------------------------------------------------------------------------------------------------------+", ] ); diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 4034800c30cb..21e40d6ede61 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3544,7 +3544,8 @@ async fn test_distribute_sort_memtable() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [id@0 ASC NULLS LAST]", " SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true]", - " DataSourceExec: partitions=3, partition_sizes=[34, 33, 33]", + " YieldStreamExec child=DataSourceExec", + " DataSourceExec: partitions=3, partition_sizes=[34, 33, 33]", ]; plans_matches_expected!(expected, physical_plan); diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 70e94227cfad..9a9ec19a7d9b 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -613,7 +613,8 @@ async fn test_physical_plan_display_indent() { " CoalesceBatchesExec: target_batch_size=4096", " FilterExec: c12@1 < 10", " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true", + " YieldStreamExec child=DataSourceExec", + " DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true", ]; let normalizer = ExplainNormalizer::new(); diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index 718907fc0b5b..2f08106ba120 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -20,13 +20,15 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; +use crate::execution_plan::CardinalityEffect; +use crate::execution_plan::CardinalityEffect::Equal; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, SendableRecordBatchStream, }; use arrow::record_batch::RecordBatch; use arrow_schema::Schema; -use datafusion_common::Result; +use datafusion_common::{Result, Statistics}; use datafusion_execution::TaskContext; use futures::Stream; @@ -84,7 +86,10 @@ impl Stream for YieldStream { Poll::Ready(Some(Ok(batch))) } } - other => other, + other => { + this.batches_processed = 0; // Reset count + other + } } } } @@ -158,6 +163,27 @@ impl ExecutionPlan for YieldStreamExec { let yield_stream = YieldStream::new(child_stream); Ok(Box::pin(yield_stream)) } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_none() { + self.child.partition_statistics(partition) + } else { + Ok(Statistics::new_unknown(&self.schema())) + } + } + + fn maintains_input_order(&self) -> Vec { + // YieldStreamExec does not change the order of the input data + self.child.maintains_input_order() + } + + fn supports_limit_pushdown(&self) -> bool { + true + } + + fn cardinality_effect(&self) -> CardinalityEffect { + Equal + } } #[cfg(test)] From 97c1bb7723d31aa871520877fcebaf6d5013dbd6 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 2 Jun 2025 23:08:12 +0800 Subject: [PATCH 23/74] fix --- .../core/tests/dataframe/.mod.rs.pending-snap | 787 ------------------ 1 file changed, 787 deletions(-) delete mode 100644 datafusion/core/tests/dataframe/.mod.rs.pending-snap diff --git a/datafusion/core/tests/dataframe/.mod.rs.pending-snap b/datafusion/core/tests/dataframe/.mod.rs.pending-snap deleted file mode 100644 index 36509e904b10..000000000000 --- a/datafusion/core/tests/dataframe/.mod.rs.pending-snap +++ /dev/null @@ -1,787 +0,0 @@ -{"run_id":"1748873147-503907000","line":571,"new":{"module_name":"core_integration__dataframe","snapshot_name":"aggregate_with_pk","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":571,"expression":"physical_plan_to_string(&df).await"},"snapshot":"AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]\n YieldStreamExec child=DataSourceExec\n DataSourceExec: partitions=1, partition_sizes=[1]"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]\n DataSourceExec: partitions=1, partition_sizes=[1]"}} -{"run_id":"1748873175-977737000","line":571,"new":null,"old":null} -{"run_id":"1748873175-977737000","line":582,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4557,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5490,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2022,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":511,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1920,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5509,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5548,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5561,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5417,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2029,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5435,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":480,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":448,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5450,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3221,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3203,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3266,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1197,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1243,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1170,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3506,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1219,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1224,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3517,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3598,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3239,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3551,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3610,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":342,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2310,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3302,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3564,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3167,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3185,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5334,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":874,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3080,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5352,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":571,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3127,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":582,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5367,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":613,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5647,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":663,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5658,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":715,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5720,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":678,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":628,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":728,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5725,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5755,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5767,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5696,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5783,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4878,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":756,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1389,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1405,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":839,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2716,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4973,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2635,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5042,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4985,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5003,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5055,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5075,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5011,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2755,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5084,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2857,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4909,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4922,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4942,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2883,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4952,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2681,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1042,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1129,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":803,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5943,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5971,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1890,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5988,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1061,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":6066,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3337,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4508,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1096,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4474,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1013,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4053,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4153,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3810,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5583,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4181,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4074,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3832,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3851,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3630,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3876,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3646,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3930,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5615,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4205,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3898,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3952,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3760,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4001,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4421,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4022,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4348,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3672,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3788,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4379,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4249,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4282,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3700,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1986,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1710,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3429,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":4293,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":3380,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1532,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1447,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1830,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2479,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2422,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1674,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":931,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1574,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1587,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1470,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1603,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1846,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2790,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2790,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748873250-690022000","line":5148,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2364,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2538,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1745,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1758,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2575,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_sort","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2575,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.b, count(*) |\n| | Sort: count(Int64(1)) AS count(*) AS count(*) ASC NULLS LAST |\n| | Projection: t1.b, count(Int64(1)) AS count(*), count(Int64(1)) |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | ProjectionExec: expr=[b@0 as b, count(*)@1 as count(*)] |\n| | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] |\n| | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] |\n| | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.b, count(*) |\n| | Sort: count(Int64(1)) AS count(*) AS count(*) ASC NULLS LAST |\n| | Projection: t1.b, count(Int64(1)) AS count(*), count(Int64(1)) |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | ProjectionExec: expr=[b@0 as b, count(*)@1 as count(*)] |\n| | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] |\n| | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] |\n| | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748873250-690022000","line":1774,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":5170,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1493,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":2912,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_where_scalar_subquery","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2912,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.a, t1.b |\n| | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) |\n| | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true |\n| | Left Join: t1.a = __scalar_sq_1.a |\n| | TableScan: t1 projection=[a, b] |\n| | SubqueryAlias: __scalar_sq_1 |\n| | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true |\n| | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] |\n| | TableScan: t2 projection=[a] |\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\n| | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.a, t1.b |\n| | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) |\n| | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true |\n| | Left Join: t1.a = __scalar_sq_1.a |\n| | TableScan: t1 projection=[a, b] |\n| | SubqueryAlias: __scalar_sq_1 |\n| | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true |\n| | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] |\n| | TableScan: t2 projection=[a] |\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\n| | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748873250-690022000","line":1618,"new":null,"old":null} -{"run_id":"1748873250-690022000","line":1789,"new":null,"old":null} -{"run_id":"1748873447-22072000","line":2575,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_sort","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2575,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.b, count(*) |\n| | Sort: count(Int64(1)) AS count(*) AS count(*) ASC NULLS LAST |\n| | Projection: t1.b, count(Int64(1)) AS count(*), count(Int64(1)) |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | ProjectionExec: expr=[b@0 as b, count(*)@1 as count(*)] |\n| | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] |\n| | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] |\n| | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.b, count(*) |\n| | Sort: count(Int64(1)) AS count(*) AS count(*) ASC NULLS LAST |\n| | Projection: t1.b, count(Int64(1)) AS count(*), count(Int64(1)) |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | ProjectionExec: expr=[b@0 as b, count(*)@1 as count(*)] |\n| | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] |\n| | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] |\n| | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748873613-663955000","line":2575,"new":null,"old":null} -{"run_id":"1748873613-663955000","line":2602,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_sort-2","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2602,"expression":"pretty_format_batches(&df_results).unwrap()"},"snapshot":"+---------------+--------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+--------------------------------------------------------------------------------+\n| logical_plan | Sort: count(*) ASC NULLS LAST |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] |\n| | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+--------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+--------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+--------------------------------------------------------------------------------+\n| logical_plan | Sort: count(*) ASC NULLS LAST |\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] |\n| | TableScan: t1 projection=[b] |\n| physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] |\n| | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+--------------------------------------------------------------------------------+"}} -{"run_id":"1748873676-404273000","line":2575,"new":null,"old":null} -{"run_id":"1748873676-404273000","line":2602,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2022,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5493,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4560,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":511,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1920,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5512,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5551,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5564,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2029,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5420,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5438,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":480,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5453,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":448,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3206,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3224,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3269,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1243,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1197,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1170,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3509,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1219,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1224,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3601,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3554,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3613,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3520,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3242,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2310,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":342,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3567,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3305,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3188,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3170,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":874,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5337,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3083,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5355,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5370,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":571,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":613,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3130,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":582,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5650,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":628,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":715,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5661,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":663,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":728,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5723,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5758,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5699,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5728,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":678,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5770,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5786,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":756,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1389,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4881,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1405,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":839,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2719,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2758,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4976,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":803,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5045,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2638,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2575,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2602,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5058,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4988,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5078,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5006,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2860,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4912,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5087,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5014,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4925,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2886,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1129,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4945,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4955,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1042,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2684,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5974,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5946,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1890,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5991,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4477,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":6069,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4511,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3340,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1061,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1096,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1013,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4056,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4156,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5618,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5586,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4077,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3879,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3633,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3813,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4184,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3763,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3901,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3933,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3835,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3649,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3791,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4004,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3955,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3854,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4208,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4252,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4424,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4025,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4351,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4285,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3675,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4382,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":4296,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3703,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1710,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1986,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1532,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3432,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2422,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1447,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2479,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":3383,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":931,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1574,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1587,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1674,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1603,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1830,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5151,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2364,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1470,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2538,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":5173,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1745,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1758,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1846,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1774,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748873683-425586000","line":2915,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_where_scalar_subquery","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2915,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.a, t1.b |\n| | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) |\n| | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true |\n| | Left Join: t1.a = __scalar_sq_1.a |\n| | TableScan: t1 projection=[a, b] |\n| | SubqueryAlias: __scalar_sq_1 |\n| | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true |\n| | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] |\n| | TableScan: t2 projection=[a] |\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\n| | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: t1.a, t1.b |\n| | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) |\n| | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true |\n| | Left Join: t1.a = __scalar_sq_1.a |\n| | TableScan: t1 projection=[a, b] |\n| | SubqueryAlias: __scalar_sq_1 |\n| | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true |\n| | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] |\n| | TableScan: t2 projection=[a] |\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\n| | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] |\n| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | CoalesceBatchesExec: target_batch_size=8192 |\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\n| | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748873683-425586000","line":1493,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1618,"new":null,"old":null} -{"run_id":"1748873683-425586000","line":1789,"new":null,"old":null} -{"run_id":"1748873859-597000000","line":2915,"new":null,"old":null} -{"run_id":"1748873859-597000000","line":2975,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5499,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2022,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4566,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":511,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5518,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5557,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5570,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1920,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5426,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2029,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5444,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5459,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":480,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":448,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3230,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3212,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1170,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3275,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1197,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1243,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1219,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1224,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3515,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3607,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3560,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3526,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3619,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2310,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3248,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3573,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":342,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3194,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3176,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3311,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":874,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5343,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3089,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5361,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3136,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":571,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5376,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":663,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":582,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":613,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":715,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5656,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":678,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":628,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5667,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5729,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":728,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5764,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5734,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4887,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5776,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5792,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5705,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1389,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1405,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":756,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":839,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2719,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2638,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4982,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2860,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4994,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5012,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5020,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2886,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2758,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5051,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4918,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5064,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2575,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2684,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2602,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4931,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4951,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5084,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2915,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1129,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4961,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5093,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1042,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":803,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5980,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5952,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5997,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2975,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":6075,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4517,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4483,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3346,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1013,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1061,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4062,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5592,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4162,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3819,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1096,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3841,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4083,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3639,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3860,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3885,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3655,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5624,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4190,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3939,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3907,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3769,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3961,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4010,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4031,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3797,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4357,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4430,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3681,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4258,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4388,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4214,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4291,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3709,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":4302,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1986,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1890,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1710,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1532,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1447,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3389,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2422,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":3438,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":931,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2479,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1830,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1574,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1587,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1674,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1470,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2538,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1603,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2364,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5157,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":5179,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1846,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1745,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1758,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1493,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1774,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748873866-668387000","line":1618,"new":null,"old":null} -{"run_id":"1748873866-668387000","line":1789,"new":null,"old":null} -{"run_id":"1748874092-266512000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748874158-473190000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748874308-762369000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748874553-675455000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748874806-77292000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748874842-394238000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748874912-217000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748874955-173216000","line":5503,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1920,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2022,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4570,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":511,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5522,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5561,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5574,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2029,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5430,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5448,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":480,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5463,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":448,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3234,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3216,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3279,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1243,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1197,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1170,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1219,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1224,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3519,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3611,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3564,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3530,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2310,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3252,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3623,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3577,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":342,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3198,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3180,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3315,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5347,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":874,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5365,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5380,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3093,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3140,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":571,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":582,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5660,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":663,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":613,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5671,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":715,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":678,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5733,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":628,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5738,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":728,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5709,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5768,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4891,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5780,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5796,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":756,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1389,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1405,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":839,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2719,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2864,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4986,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2638,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4998,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5016,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5024,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2890,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5055,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4922,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2575,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2602,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4935,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4955,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5068,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5088,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4965,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2758,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2919,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5097,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2684,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1129,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1042,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5984,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5956,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2979,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":6001,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":6079,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1890,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4487,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3350,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4521,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1061,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1013,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5596,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4166,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4066,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3823,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3845,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1096,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3643,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3864,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4194,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4087,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3659,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3943,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5628,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3965,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4218,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3889,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4014,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3773,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3911,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":803,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4361,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3801,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4035,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4392,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4434,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4262,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3685,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4295,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1986,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3713,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":4306,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1710,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1532,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1447,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3442,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2422,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1830,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":3393,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2479,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":931,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1674,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1470,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2364,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5161,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1574,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1846,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1587,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2538,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":5183,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1603,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1745,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1758,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1493,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1774,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748874955-173216000","line":1618,"new":null,"old":null} -{"run_id":"1748874955-173216000","line":1789,"new":null,"old":null} -{"run_id":"1748875050-86297000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748875102-610079000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748875163-578276000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748875364-204310000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \\\"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\\\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted]|\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748875457-236822000","line":2793,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2793,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748875515-685334000","line":2795,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2795,"expression":"pretty_format_batches(&sql_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748875568-55246000","line":2795,"new":null,"old":null} -{"run_id":"1748875568-55246000","line":2831,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window-2","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2831,"expression":"pretty_format_batches(&df_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748875609-922318000","line":2793,"new":null,"old":null} -{"run_id":"1748875609-922318000","line":2832,"new":{"module_name":"core_integration__dataframe","snapshot_name":"count_wildcard_on_window-2","metadata":{"source":"datafusion/core/tests/dataframe/mod.rs","assertion_line":2832,"expression":"pretty_format_batches(&df_results).unwrap()"},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec child=DataSourceExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"},"old":{"module_name":"core_integration__dataframe","metadata":{},"snapshot":"+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| plan_type | plan |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\n| | TableScan: t1 projection=[a] |\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\n| | YieldStreamExec |\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\n| | |\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"}} -{"run_id":"1748875647-407491000","line":2793,"new":null,"old":null} -{"run_id":"1748875647-407491000","line":2832,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5499,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5518,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5557,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5570,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":511,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2022,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1920,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4566,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5426,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2029,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5444,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5459,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":480,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":448,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3230,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3212,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3275,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1243,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1197,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1219,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1224,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3515,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1170,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3526,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3607,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3560,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2310,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3619,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3248,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3573,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3311,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":342,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3176,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3194,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3136,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":874,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":571,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5343,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":582,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3089,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":613,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5361,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":628,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5729,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5734,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5376,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":715,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5656,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5667,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":663,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":728,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":678,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4887,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5705,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5764,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":756,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5776,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5792,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1389,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1405,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2719,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2575,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2602,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":839,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2862,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2758,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2638,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2888,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4982,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4994,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5012,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4918,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5051,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5020,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2793,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4931,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4951,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5064,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5084,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4961,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2828,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2684,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5093,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1129,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1042,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5952,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2917,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5980,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5997,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3346,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":6075,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1096,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1061,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2976,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4483,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4517,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1013,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4062,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5592,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4083,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4162,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5624,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1890,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3639,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4190,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3819,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3769,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3885,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3841,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3907,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3860,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3655,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3797,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4214,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3939,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4010,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4357,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3961,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4031,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3681,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4388,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4430,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4258,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4291,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3709,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":803,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":4302,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1710,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1986,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1447,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2422,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1532,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2479,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3389,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1674,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1830,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1574,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1587,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":931,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1603,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":3438,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1470,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2364,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":2538,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5157,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1745,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1758,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1846,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":5179,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1774,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1493,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1618,"new":null,"old":null} -{"run_id":"1748876842-95102000","line":1789,"new":null,"old":null} From 5c3a14c14d6c3b06e34a5f1cd6068500f6b4fd24 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Tue, 3 Jun 2025 08:52:24 +0800 Subject: [PATCH 24/74] remove buffer --- datafusion/physical-plan/src/yield_stream.rs | 37 +++++++------------- 1 file changed, 13 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index 2f08106ba120..a2a165ca62ca 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -30,7 +30,7 @@ use arrow::record_batch::RecordBatch; use arrow_schema::Schema; use datafusion_common::{Result, Statistics}; use datafusion_execution::TaskContext; -use futures::Stream; +use futures::{Stream, StreamExt}; /// Number of batches to yield before voluntarily returning Pending. /// This allows long-running operators to periodically yield control @@ -43,7 +43,6 @@ const YIELD_BATCHES: usize = 64; pub struct YieldStream { inner: SendableRecordBatchStream, batches_processed: usize, - buffer: Option>, } impl YieldStream { @@ -51,7 +50,6 @@ impl YieldStream { Self { inner, batches_processed: 0, - buffer: None, } } } @@ -64,32 +62,23 @@ impl Stream for YieldStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - let this = &mut *self; - - if let Some(batch) = this.buffer.take() { - return Poll::Ready(Some(batch)); + if self.batches_processed >= YIELD_BATCHES { + self.batches_processed = 0; + cx.waker().wake_by_ref(); + return Poll::Pending; } - // Instead of `poll_next_unpin`, use `Pin::new(&mut this.inner).poll_next(cx)` - match Pin::new(&mut this.inner).poll_next(cx) { + match self.inner.poll_next_unpin(cx) { Poll::Ready(Some(Ok(batch))) => { - this.batches_processed += 1; - if this.batches_processed >= YIELD_BATCHES { - this.batches_processed = 0; - // We need to buffer the batch when we return Poll::Pending, - // so that we can return it on the next poll. - // Otherwise, the next poll will miss the batch and return None. - this.buffer = Some(Ok(batch)); - cx.waker().wake_by_ref(); - Poll::Pending - } else { - Poll::Ready(Some(Ok(batch))) - } + self.batches_processed += 1; + Poll::Ready(Some(Ok(batch))) } - other => { - this.batches_processed = 0; // Reset count - other + Poll::Pending => { + self.batches_processed = 0; + Poll::Pending } + + other => other, } } } From 97923b80f4c67b73127815b7f112d208817ca1fc Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Tue, 3 Jun 2025 09:30:44 +0800 Subject: [PATCH 25/74] address comments --- .../src/wrap_leaves_cancellation.rs | 77 ++++++++++--------- .../sqllogictest/test_files/explain_tree.slt | 28 +++---- 2 files changed, 54 insertions(+), 51 deletions(-) diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index 87a793e29787..887c8a7b8dc9 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -23,52 +23,53 @@ use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::ExecutionPlan; use std::fmt::{Debug, Formatter}; use std::sync::Arc; +use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion}; -/// `WrapLeaves` is a `PhysicalOptimizerRule` that traverses a physical plan -/// and, for every operator whose `emission_type` is `Final`, wraps its direct -/// children inside a `YieldStreamExec`. This ensures that pipeline‐breaking -/// operators (i.e. those with `Final` emission) have a “yield point” immediately -/// upstream, without having to wait until the leaves. +/// WrapLeaves is a PhysicalOptimizerRule that finds every +/// pipeline‐breaking node (emission_type == Final) and then +/// wraps all of its leaf children in YieldStreamExec. pub struct WrapLeaves {} impl WrapLeaves { - /// Create a new instance of the WrapLeaves rule. pub fn new() -> Self { Self {} } - /// Recursively walk the plan: - /// - If `plan.children_any().is_empty()`, it’s a leaf, so wrap it. - /// - Otherwise, recurse into its children, rebuild the node with - /// `with_new_children_any(...)`, and return that. - #[allow(clippy::only_used_in_recursion)] - fn wrap_recursive( - &self, + /// This function is called on every plan node during transform_down(). + /// If the node is a leaf (no children), we wrap it in a new YieldStreamExec + /// and stop recursing further under that branch (TreeNodeRecursion::Jump). + fn wrap_leaves( plan: Arc, - has_pipeline_breaking_above: bool, - ) -> Result> { - let children = plan.children(); + ) -> Result>> { + if plan.children().is_empty() { + // Leaf: wrap it in YieldStreamExec, and do not descend further + let wrapped = Arc::new(YieldStreamExec::new(plan)); + Ok(Transformed::new(wrapped, /* changed */ true, TreeNodeRecursion::Jump)) + } else { + // Not a leaf: leave unchanged and keep recursing + Ok(Transformed::no(plan)) + } + } + /// This function is called on every plan node during transform_down(). + /// + /// If this node itself is a pipeline breaker (emission_type == Final), + /// we perform a second pass of transform_down with wrap_leaves. Then we + /// set TreeNodeRecursion::Jump so that we do not descend any deeper under + /// this subtree (we’ve already wrapped its leaves). + fn wrap_leaves_of_pipeline_breakers( + plan: Arc, + ) -> Result>> { let is_pipeline_breaker = plan.properties().emission_type == EmissionType::Final; - let should_wrap = has_pipeline_breaking_above; - - if children.is_empty() { - // Leaf node: wrap it in `YieldStreamExec` - if should_wrap { - Ok(Arc::new(YieldStreamExec::new(plan))) - } else { - Ok(plan) - } + if is_pipeline_breaker { + // Transform all leaf descendants of this node by calling wrap_leaves + let mut transformed = plan.transform_down(Self::wrap_leaves)?; + // Once we’ve handled the leaves of this subtree, we skip deeper recursion + transformed.tnr = TreeNodeRecursion::Jump; + Ok(transformed) } else { - let mut new_children = Vec::with_capacity(children.len()); - for child in children { - let new_child = self.wrap_recursive( - Arc::clone(child), - has_pipeline_breaking_above || is_pipeline_breaker, - )?; - new_children.push(new_child); - } - Ok(plan.with_new_children(new_children)?) + // Not a pipeline breaker: do nothing here, let transform_down recurse + Ok(Transformed::no(plan)) } } } @@ -90,17 +91,19 @@ impl PhysicalOptimizerRule for WrapLeaves { "wrap_leaves" } - /// Apply the rule by calling `wrap_recursive` on the root plan. fn optimize( &self, plan: Arc, _config: &ConfigOptions, ) -> Result> { - self.wrap_recursive(plan, false) + // We run a top‐level transform_down: for every node, call wrap_leaves_of_pipeline_breakers. + // If a node is a pipeline breaker, we then wrap all of its leaf children in YieldStreamExec. + plan.transform_down(Self::wrap_leaves_of_pipeline_breakers) + .map(|t| t.data) } - /// Since we only add `YieldStreamExec` wrappers (which preserve schema), schema_check remains true. fn schema_check(&self) -> bool { + // Wrapping a leaf in YieldStreamExec preserves the schema, so we’re fine true } } diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index c1ae8bc184c6..c847e67bdb91 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -243,7 +243,7 @@ physical_plan 43)│ RoundRobinBatch(4) │ 44)└─────────────┬─────────────┘ 45)┌─────────────┴─────────────┐ -46)│ yield_stream_exec │ +46)│ YieldStreamExec │ 47)│ -------------------- │ 48)│ YieldStreamExec child: │ 49)│ DataSourceExec │ @@ -773,7 +773,7 @@ physical_plan 19)│ FOLLOWING │ 20)└─────────────┬─────────────┘ 21)┌─────────────┴─────────────┐ -22)│ yield_stream_exec │ +22)│ YieldStreamExec │ 23)│ -------------------- │ 24)│ YieldStreamExec child: │ 25)│ DataSourceExec │ @@ -826,7 +826,7 @@ physical_plan 31)│ v1: value │ 32)└─────────────┬─────────────┘ 33)┌─────────────┴─────────────┐ -34)│ yield_stream_exec │ +34)│ YieldStreamExec │ 35)│ -------------------- │ 36)│ YieldStreamExec child: │ 37)│ LazyMemoryExec │ @@ -878,7 +878,7 @@ physical_plan 29)│ FOLLOWING │ 30)└─────────────┬─────────────┘ 31)┌─────────────┴─────────────┐ -32)│ yield_stream_exec │ +32)│ YieldStreamExec │ 33)│ -------------------- │ 34)│ YieldStreamExec child: │ 35)│ DataSourceExec │ @@ -901,7 +901,7 @@ physical_plan 04)│string_col@1 ASC NULLS LAST│ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ yield_stream_exec │ +07)│ YieldStreamExec │ 08)│ -------------------- │ 09)│ YieldStreamExec child: │ 10)│ DataSourceExec │ @@ -926,7 +926,7 @@ physical_plan 06)│string_col@1 ASC NULLS LAST│ 07)└─────────────┬─────────────┘ 08)┌─────────────┴─────────────┐ -09)│ yield_stream_exec │ +09)│ YieldStreamExec │ 10)│ -------------------- │ 11)│ YieldStreamExec child: │ 12)│ DataSourceExec │ @@ -1041,7 +1041,7 @@ physical_plan 61)│ int_col@0 DESC │ 62)└─────────────┬─────────────┘ 63)┌─────────────┴─────────────┐ -64)│ yield_stream_exec │ +64)│ YieldStreamExec │ 65)│ -------------------- │ 66)│ YieldStreamExec child: │ 67)│ DataSourceExec │ @@ -1375,7 +1375,7 @@ physical_plan 04)│ join_type: LeftSemi │ │ 05)└─────────────┬─────────────┘ │ 06)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -07)│ yield_stream_exec ││ ProjectionExec │ +07)│ YieldStreamExec ││ ProjectionExec │ 08)│ -------------------- ││ │ 09)│ YieldStreamExec child: ││ │ 10)│ DataSourceExec ││ │ @@ -1405,7 +1405,7 @@ physical_plan 34)-----------------------------│ RoundRobinBatch(4) │ 35)-----------------------------└─────────────┬─────────────┘ 36)-----------------------------┌─────────────┴─────────────┐ -37)-----------------------------│ yield_stream_exec │ +37)-----------------------------│ YieldStreamExec │ 38)-----------------------------│ -------------------- │ 39)-----------------------------│ YieldStreamExec child: │ 40)-----------------------------│ DataSourceExec │ @@ -1426,7 +1426,7 @@ physical_plan 02)│ CrossJoinExec ├──────────────┐ 03)└─────────────┬─────────────┘ │ 04)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -05)│ yield_stream_exec ││ RepartitionExec │ +05)│ YieldStreamExec ││ RepartitionExec │ 06)│ -------------------- ││ -------------------- │ 07)│ YieldStreamExec child: ││ partition_count(in->out): │ 08)│ DataSourceExec ││ 1 -> 4 │ @@ -1435,7 +1435,7 @@ physical_plan 11)│ ││ RoundRobinBatch(4) │ 12)└─────────────┬─────────────┘└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -14)│ DataSourceExec ││ yield_stream_exec │ +14)│ DataSourceExec ││ YieldStreamExec │ 15)│ -------------------- ││ -------------------- │ 16)│ files: 1 ││ YieldStreamExec child: │ 17)│ format: csv ││ DataSourceExec │ @@ -1467,7 +1467,7 @@ physical_plan 09)│ c1@0 ASC ││ c1@0 ASC │ 10)└─────────────┬─────────────┘└─────────────┬─────────────┘ 11)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -12)│ yield_stream_exec ││ yield_stream_exec │ +12)│ YieldStreamExec ││ YieldStreamExec │ 13)│ -------------------- ││ -------------------- │ 14)│ YieldStreamExec child: ││ YieldStreamExec child: │ 15)│ DataSourceExec ││ DataSourceExec │ @@ -1587,7 +1587,7 @@ physical_plan 55)│ mode: Partial ││ mode: Partial │ 56)└─────────────┬─────────────┘└─────────────┬─────────────┘ 57)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -58)│ yield_stream_exec ││ yield_stream_exec │ +58)│ YieldStreamExec ││ YieldStreamExec │ 59)│ -------------------- ││ -------------------- │ 60)│ YieldStreamExec child: ││ YieldStreamExec child: │ 61)│ DataSourceExec ││ DataSourceExec │ @@ -2078,7 +2078,7 @@ physical_plan 51)│ predicate: a > 3 │ 52)└─────────────┬─────────────┘ 53)┌─────────────┴─────────────┐ -54)│ yield_stream_exec │ +54)│ YieldStreamExec │ 55)│ -------------------- │ 56)│ YieldStreamExec child: │ 57)│ DataSourceExec │ From 118f801939506f933e02e17bd0c770360c8dadc1 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Tue, 3 Jun 2025 09:34:38 +0800 Subject: [PATCH 26/74] fmt --- .../physical-optimizer/src/wrap_leaves_cancellation.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index 887c8a7b8dc9..63311b1abd7c 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -17,13 +17,13 @@ use crate::PhysicalOptimizerRule; use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion}; use datafusion_common::Result; use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::ExecutionPlan; use std::fmt::{Debug, Formatter}; use std::sync::Arc; -use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion}; /// WrapLeaves is a PhysicalOptimizerRule that finds every /// pipeline‐breaking node (emission_type == Final) and then @@ -44,7 +44,11 @@ impl WrapLeaves { if plan.children().is_empty() { // Leaf: wrap it in YieldStreamExec, and do not descend further let wrapped = Arc::new(YieldStreamExec::new(plan)); - Ok(Transformed::new(wrapped, /* changed */ true, TreeNodeRecursion::Jump)) + Ok(Transformed::new( + wrapped, + /* changed */ true, + TreeNodeRecursion::Jump, + )) } else { // Not a leaf: leave unchanged and keep recursing Ok(Transformed::no(plan)) From e7a678aec2bd71d8ea745c38df2cde5084ccf3bb Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Tue, 3 Jun 2025 21:56:07 +0800 Subject: [PATCH 27/74] Fix test --- datafusion/physical-plan/src/yield_stream.rs | 6 ++ datafusion/proto/proto/datafusion.proto | 5 + datafusion/proto/src/generated/pbjson.rs | 105 +++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 9 +- datafusion/proto/src/physical_plan/mod.rs | 46 ++++++++ 5 files changed, 170 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index a2a165ca62ca..fb2629cc352a 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -114,6 +114,12 @@ impl DisplayAs for YieldStreamExec { } } +impl YieldStreamExec { + pub fn input(&self) -> &Arc { + &self.child + } +} + impl ExecutionPlan for YieldStreamExec { fn name(&self) -> &str { "YieldStreamExec" diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 4c8b6c588d94..460a1ae81497 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -726,6 +726,7 @@ message PhysicalPlanNode { ParquetSinkExecNode parquet_sink = 29; UnnestExecNode unnest = 30; JsonScanExecNode json_scan = 31; + YieldStreamExecNode yield_stream = 32; } } @@ -1033,6 +1034,10 @@ message AvroScanExecNode { FileScanExecConf base_conf = 1; } +message YieldStreamExecNode { + PhysicalPlanNode input = 1; +} + enum PartitionMode { COLLECT_LEFT = 0; PARTITIONED = 1; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 932422944508..64ca87b895c4 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -15796,6 +15796,9 @@ impl serde::Serialize for PhysicalPlanNode { physical_plan_node::PhysicalPlanType::JsonScan(v) => { struct_ser.serialize_field("jsonScan", v)?; } + physical_plan_node::PhysicalPlanType::YieldStream(v) => { + struct_ser.serialize_field("yieldStream", v)?; + } } } struct_ser.end() @@ -15854,6 +15857,8 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "unnest", "json_scan", "jsonScan", + "yield_stream", + "yieldStream", ]; #[allow(clippy::enum_variant_names)] @@ -15888,6 +15893,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { ParquetSink, Unnest, JsonScan, + YieldStream, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -15939,6 +15945,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "parquetSink" | "parquet_sink" => Ok(GeneratedField::ParquetSink), "unnest" => Ok(GeneratedField::Unnest), "jsonScan" | "json_scan" => Ok(GeneratedField::JsonScan), + "yieldStream" | "yield_stream" => Ok(GeneratedField::YieldStream), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -16169,6 +16176,13 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { return Err(serde::de::Error::duplicate_field("jsonScan")); } physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::JsonScan) +; + } + GeneratedField::YieldStream => { + if physical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("yieldStream")); + } + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::YieldStream) ; } } @@ -22665,3 +22679,94 @@ impl<'de> serde::Deserialize<'de> for WindowNode { deserializer.deserialize_struct("datafusion.WindowNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for YieldStreamExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.input.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.YieldStreamExecNode", len)?; + if let Some(v) = self.input.as_ref() { + struct_ser.serialize_field("input", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for YieldStreamExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "input", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Input, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "input" => Ok(GeneratedField::Input), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = YieldStreamExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.YieldStreamExecNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut input__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Input => { + if input__.is_some() { + return Err(serde::de::Error::duplicate_field("input")); + } + input__ = map_.next_value()?; + } + } + } + Ok(YieldStreamExecNode { + input: input__, + }) + } + } + deserializer.deserialize_struct("datafusion.YieldStreamExecNode", FIELDS, GeneratedVisitor) + } +} diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index c2f4e93cef6a..f289d42c69c8 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1048,7 +1048,7 @@ pub mod table_reference { pub struct PhysicalPlanNode { #[prost( oneof = "physical_plan_node::PhysicalPlanType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32" )] pub physical_plan_type: ::core::option::Option, } @@ -1118,6 +1118,8 @@ pub mod physical_plan_node { Unnest(::prost::alloc::boxed::Box), #[prost(message, tag = "31")] JsonScan(super::JsonScanExecNode), + #[prost(message, tag = "32")] + YieldStream(::prost::alloc::boxed::Box), } } #[derive(Clone, PartialEq, ::prost::Message)] @@ -1572,6 +1574,11 @@ pub struct AvroScanExecNode { pub base_conf: ::core::option::Option, } #[derive(Clone, PartialEq, ::prost::Message)] +pub struct YieldStreamExecNode { + #[prost(message, optional, boxed, tag = "1")] + pub input: ::core::option::Option<::prost::alloc::boxed::Box>, +} +#[derive(Clone, PartialEq, ::prost::Message)] pub struct HashJoinExecNode { #[prost(message, optional, boxed, tag = "1")] pub left: ::core::option::Option<::prost::alloc::boxed::Box>, diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 7a85a2a8efbd..020baa09c195 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -89,6 +89,7 @@ use datafusion_common::config::TableParquetOptions; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; +use datafusion::physical_plan::yield_stream::YieldStreamExec; use prost::bytes::BufMut; use prost::Message; @@ -324,6 +325,14 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { runtime, extension_codec, ), + + PhysicalPlanType::YieldStream(yield_stream) => self + .try_into_yield_stream_physical_plan( + yield_stream, + registry, + runtime, + extension_codec, + ), } } @@ -513,6 +522,13 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ); } + if let Some(exec) = plan.downcast_ref::() { + return protobuf::PhysicalPlanNode::try_from_yield_stream_exec( + exec, + extension_codec, + ); + } + let mut buf: Vec = vec![]; match extension_codec.try_encode(Arc::clone(&plan_clone), &mut buf) { Ok(_) => { @@ -1769,6 +1785,18 @@ impl protobuf::PhysicalPlanNode { ))) } + fn try_into_yield_stream_physical_plan( + &self, + field_stream: &protobuf::YieldStreamExecNode, + registry: &dyn FunctionRegistry, + runtime: &RuntimeEnv, + extension_codec: &dyn PhysicalExtensionCodec, + ) -> Result> { + let input = + into_physical_plan(&field_stream.input, registry, runtime, extension_codec)?; + Ok(Arc::new(YieldStreamExec::new(input))) + } + fn try_from_explain_exec( exec: &ExplainExec, _extension_codec: &dyn PhysicalExtensionCodec, @@ -2748,6 +2776,24 @@ impl protobuf::PhysicalPlanNode { ))), }) } + + fn try_from_yield_stream_exec( + exec: &YieldStreamExec, + extension_codec: &dyn PhysicalExtensionCodec, + ) -> Result { + let input = protobuf::PhysicalPlanNode::try_from_physical_plan( + exec.input().to_owned(), + extension_codec, + )?; + + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::YieldStream(Box::new( + protobuf::YieldStreamExecNode { + input: Some(Box::new(input)), + }, + ))), + }) + } } pub trait AsExecutionPlan: Debug + Send + Sync + Clone { From 54260aa3758e9fe7f23acd8babf5f8fbe2a9dbc8 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Tue, 3 Jun 2025 22:03:55 +0800 Subject: [PATCH 28/74] fix --- datafusion-examples/examples/planner_api.rs | 17 ++++++++++------- datafusion/physical-plan/src/yield_stream.rs | 5 +++++ 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/datafusion-examples/examples/planner_api.rs b/datafusion-examples/examples/planner_api.rs index 55aec7b0108a..3183cda3a36d 100644 --- a/datafusion-examples/examples/planner_api.rs +++ b/datafusion-examples/examples/planner_api.rs @@ -117,18 +117,21 @@ async fn to_physical_plan_step_by_step_demo( displayable(physical_plan.as_ref()).indent(false) ); + // todo, the following code is commented out because it is actually not a right way to optimize based a already optimized plan. + // And for YieldStreamExec, it shows error when optimizing it again based on the already optimized plan which already has YieldStreamExec. + // Call the physical optimizer with an existing physical plan (in this // case the plan is already optimized, but an unoptimized plan would // typically be used in this context) // Note that this is not part of the trait but a public method // on DefaultPhysicalPlanner. Not all planners will provide this feature. - let planner = DefaultPhysicalPlanner::default(); - let physical_plan = - planner.optimize_physical_plan(physical_plan, &ctx.state(), |_, _| {})?; - println!( - "Optimized physical plan:\n\n{}\n\n", - displayable(physical_plan.as_ref()).indent(false) - ); + // let planner = DefaultPhysicalPlanner::default(); + // let physical_plan = + // planner.optimize_physical_plan(physical_plan, &ctx.state(), |_, _| {})?; + // println!( + // "Optimized physical plan:\n\n{}\n\n", + // displayable(physical_plan.as_ref()).indent(false) + // ); Ok(()) } diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index fb2629cc352a..537547d3744c 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -145,6 +145,11 @@ impl ExecutionPlan for YieldStreamExec { self: Arc, children: Vec>, ) -> Result> { + if children.len() != 1 { + return Err(datafusion_common::DataFusionError::Internal( + "YieldStreamExec requires exactly one child".to_string(), + )); + } // Use Arc::clone on children[0] rather than calling clone() directly Ok(Arc::new(YieldStreamExec::new(Arc::clone(&children[0])))) } From cb344af8c90785f55d20f31f022a876c033c00dc Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Tue, 3 Jun 2025 22:04:00 +0800 Subject: [PATCH 29/74] fix --- datafusion-examples/examples/planner_api.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion-examples/examples/planner_api.rs b/datafusion-examples/examples/planner_api.rs index 3183cda3a36d..051d9a2d98e2 100644 --- a/datafusion-examples/examples/planner_api.rs +++ b/datafusion-examples/examples/planner_api.rs @@ -18,7 +18,6 @@ use datafusion::error::Result; use datafusion::logical_expr::LogicalPlan; use datafusion::physical_plan::displayable; -use datafusion::physical_planner::DefaultPhysicalPlanner; use datafusion::prelude::*; /// This example demonstrates the process of converting logical plan From 89a4e93d309490beef8a8be69df77565ede7e8de Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Tue, 3 Jun 2025 22:33:48 +0800 Subject: [PATCH 30/74] fix slt --- datafusion/sqllogictest/test_files/avro.slt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/avro.slt b/datafusion/sqllogictest/test_files/avro.slt index 4573af1d59b1..4da87511dced 100644 --- a/datafusion/sqllogictest/test_files/avro.slt +++ b/datafusion/sqllogictest/test_files/avro.slt @@ -256,7 +256,8 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]}, file_type=avro +06)----------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]}, file_type=avro # test column projection order from avro file query ITII From 896fd59371282126561dc882bc842632ca35327c Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Tue, 3 Jun 2025 23:10:14 +0800 Subject: [PATCH 31/74] fix tpch sql --- .../test_files/tpch/plans/q1.slt.part | 3 +- .../test_files/tpch/plans/q10.slt.part | 34 +++--- .../test_files/tpch/plans/q11.slt.part | 72 ++++++----- .../test_files/tpch/plans/q12.slt.part | 10 +- .../test_files/tpch/plans/q13.slt.part | 14 ++- .../test_files/tpch/plans/q14.slt.part | 12 +- .../test_files/tpch/plans/q15.slt.part | 33 ++--- .../test_files/tpch/plans/q16.slt.part | 29 +++-- .../test_files/tpch/plans/q17.slt.part | 29 +++-- .../test_files/tpch/plans/q18.slt.part | 32 ++--- .../test_files/tpch/plans/q19.slt.part | 16 +-- .../test_files/tpch/plans/q2.slt.part | 113 ++++++++++-------- .../test_files/tpch/plans/q20.slt.part | 69 ++++++----- .../test_files/tpch/plans/q21.slt.part | 56 +++++---- .../test_files/tpch/plans/q22.slt.part | 25 ++-- .../test_files/tpch/plans/q3.slt.part | 25 ++-- .../test_files/tpch/plans/q4.slt.part | 14 ++- .../test_files/tpch/plans/q5.slt.part | 52 ++++---- .../test_files/tpch/plans/q6.slt.part | 3 +- .../test_files/tpch/plans/q7.slt.part | 56 +++++---- .../test_files/tpch/plans/q8.slt.part | 70 ++++++----- .../test_files/tpch/plans/q9.slt.part | 42 ++++--- 22 files changed, 448 insertions(+), 361 deletions(-) diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index 4a6ad5eddfb7..a3e23b997633 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -57,4 +57,5 @@ physical_plan 08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] -11)--------------------DataSourceExec: 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], file_type=csv, has_header=false +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 04de9153a047..d5a801e44296 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -89,18 +89,22 @@ physical_plan 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 20)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)----------------------------------------DataSourceExec: 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], file_type=csv, 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 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -26)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -27)--------------------------CoalesceBatchesExec: target_batch_size=8192 -28)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -29)------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -31)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false -32)------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -34)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +21)----------------------------------------YieldStreamExec child=DataSourceExec +22)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +23)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +25)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)----------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +27)------------------------------------------YieldStreamExec child=DataSourceExec +28)--------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +29)--------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +31)------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +33)----------------------------------YieldStreamExec child=DataSourceExec +34)------------------------------------DataSourceExec: 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], file_type=csv, 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)------------------------YieldStreamExec child=DataSourceExec +39)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index a6225daae436..1624589c3c41 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -89,37 +89,43 @@ physical_plan 14)--------------------------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] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -17)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false -18)----------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -20)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -22)--------------------CoalesceBatchesExec: target_batch_size=8192 -23)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -24)------------------------CoalesceBatchesExec: target_batch_size=8192 -25)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -26)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -28)------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)] -29)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -30)----------CoalescePartitionsExec -31)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -32)--------------CoalesceBatchesExec: target_batch_size=8192 -33)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -34)------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -36)----------------------CoalesceBatchesExec: target_batch_size=8192 -37)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -38)--------------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -40)------------------------------DataSourceExec: 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], file_type=csv, has_header=false +17)--------------------------------YieldStreamExec child=DataSourceExec +18)----------------------------------DataSourceExec: 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], file_type=csv, 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)----------------------------------YieldStreamExec child=DataSourceExec +23)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +24)--------------------CoalesceBatchesExec: target_batch_size=8192 +25)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +26)------------------------CoalesceBatchesExec: target_batch_size=8192 +27)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +28)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)------------------------------YieldStreamExec child=DataSourceExec +30)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +31)------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)] +32)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +33)----------CoalescePartitionsExec +34)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +35)--------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +37)------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +39)----------------------CoalesceBatchesExec: target_batch_size=8192 +40)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] 41)--------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -43)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -44)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -45)------------------CoalesceBatchesExec: target_batch_size=8192 -46)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -47)----------------------CoalesceBatchesExec: target_batch_size=8192 -48)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -49)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -50)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +42)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +43)------------------------------YieldStreamExec child=DataSourceExec +44)--------------------------------DataSourceExec: 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], file_type=csv, 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)--------------------------------YieldStreamExec child=DataSourceExec +49)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +50)------------------CoalesceBatchesExec: target_batch_size=8192 +51)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +52)----------------------CoalesceBatchesExec: target_batch_size=8192 +53)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +54)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +55)----------------------------YieldStreamExec child=DataSourceExec +56)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index f7344daed8c7..ae0b59e92f24 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -72,7 +72,9 @@ physical_plan 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------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 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] -14)--------------------------DataSourceExec: 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], file_type=csv, has_header=false -15)------------------CoalesceBatchesExec: target_batch_size=8192 -16)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -17)----------------------DataSourceExec: 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], file_type=csv, has_header=false +14)--------------------------YieldStreamExec child=DataSourceExec +15)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +16)------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +18)----------------------YieldStreamExec child=DataSourceExec +19)------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 96f3bd6edf32..af290dc30671 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -67,9 +67,11 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false -16)----------------------CoalesceBatchesExec: target_batch_size=8192 -17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -18)--------------------------CoalesceBatchesExec: target_batch_size=8192 -19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -20)------------------------------DataSourceExec: 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], file_type=csv, has_header=false +15)----------------------------YieldStreamExec child=DataSourceExec +16)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false +17)----------------------CoalesceBatchesExec: target_batch_size=8192 +18)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +19)--------------------------CoalesceBatchesExec: target_batch_size=8192 +20)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +21)------------------------------YieldStreamExec child=DataSourceExec +22)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index 8d8dd68c3d7b..66ec8c578516 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -52,8 +52,10 @@ physical_plan 09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] -12)----------------------DataSourceExec: 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], file_type=csv, 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)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false +12)----------------------YieldStreamExec child=DataSourceExec +13)------------------------DataSourceExec: 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], file_type=csv, has_header=false +14)--------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +16)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------YieldStreamExec child=DataSourceExec +18)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index 0636a033b25a..a7c19c7e78ea 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -85,18 +85,21 @@ physical_plan 12)----------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 13)------------------------CoalesceBatchesExec: target_batch_size=8192 14)--------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -15)----------------------------DataSourceExec: 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], file_type=csv, has_header=false -16)--------CoalesceBatchesExec: target_batch_size=8192 -17)----------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] -18)------------CoalesceBatchesExec: target_batch_size=8192 -19)--------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -20)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false -22)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -23)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -24)----------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -26)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -27)----------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -29)--------------------------DataSourceExec: 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], file_type=csv, has_header=false +15)----------------------------YieldStreamExec child=DataSourceExec +16)------------------------------DataSourceExec: 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], file_type=csv, has_header=false +17)--------CoalesceBatchesExec: target_batch_size=8192 +18)----------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] +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)------------------YieldStreamExec child=DataSourceExec +23)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false +24)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +25)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +26)----------------CoalesceBatchesExec: target_batch_size=8192 +27)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +28)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +29)----------------------CoalesceBatchesExec: target_batch_size=8192 +30)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +31)--------------------------YieldStreamExec child=DataSourceExec +32)----------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index edc452284cf9..f3a6b9145328 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -84,16 +84,19 @@ physical_plan 17)--------------------------------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] 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -20)--------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -21)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -23)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------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) }]) -25)------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -26)--------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=csv, has_header=false -27)--------------------------CoalesceBatchesExec: target_batch_size=8192 -28)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -29)------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] -31)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], file_type=csv, has_header=false +20)--------------------------------------YieldStreamExec child=DataSourceExec +21)----------------------------------------DataSourceExec: 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], file_type=csv, 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)--------------------------------------------YieldStreamExec child=DataSourceExec +28)----------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=csv, has_header=false +29)--------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +31)------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] +33)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +34)------------------------------------YieldStreamExec child=DataSourceExec +35)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index 51a0d096428c..e6afd784083b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -61,16 +61,19 @@ physical_plan 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)--------------------DataSourceExec: 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], file_type=csv, has_header=false -12)----------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)--------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] -16)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false -18)------------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] -19)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -20)----------------CoalesceBatchesExec: target_batch_size=8192 -21)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -22)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -23)----------------------DataSourceExec: 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], file_type=csv, has_header=false +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: 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], file_type=csv, 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#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] +17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)--------------------------YieldStreamExec child=DataSourceExec +19)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false +20)------------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] +21)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +22)----------------CoalesceBatchesExec: target_batch_size=8192 +23)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +24)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +25)----------------------YieldStreamExec child=DataSourceExec +26)------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 55da5371671e..0f23fca7eaa7 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -84,17 +84,21 @@ physical_plan 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)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], file_type=csv, has_header=false -19)----------------------------CoalesceBatchesExec: target_batch_size=8192 -20)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -21)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false -22)--------------------CoalesceBatchesExec: target_batch_size=8192 -23)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -24)------------------------DataSourceExec: 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], file_type=csv, has_header=false -25)----------------CoalesceBatchesExec: target_batch_size=8192 -26)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] -27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -28)----------------------CoalesceBatchesExec: target_batch_size=8192 -29)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -30)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -31)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +18)----------------------------------YieldStreamExec child=DataSourceExec +19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], file_type=csv, has_header=false +20)----------------------------CoalesceBatchesExec: target_batch_size=8192 +21)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +22)--------------------------------YieldStreamExec child=DataSourceExec +23)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false +24)--------------------CoalesceBatchesExec: target_batch_size=8192 +25)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +26)------------------------YieldStreamExec child=DataSourceExec +27)--------------------------DataSourceExec: 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], file_type=csv, has_header=false +28)----------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] +30)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +31)----------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +33)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +34)----------------------------YieldStreamExec child=DataSourceExec +35)------------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index 3b15fb3d8e53..85aad1997312 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -74,10 +74,12 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------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, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] -11)--------------------DataSourceExec: 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], file_type=csv, has_header=false -12)------------CoalesceBatchesExec: target_batch_size=8192 -13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)----------------CoalesceBatchesExec: target_batch_size=8192 -15)------------------FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE") }, Literal { value: Utf8View("SM BOX") }, Literal { value: Utf8View("SM PACK") }, Literal { value: Utf8View("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG") }, Literal { value: Utf8View("MED BOX") }, Literal { value: Utf8View("MED PKG") }, Literal { value: Utf8View("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE") }, Literal { value: Utf8View("LG BOX") }, Literal { value: Utf8View("LG PACK") }, Literal { value: Utf8View("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 -16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=csv, has_header=false +11)--------------------YieldStreamExec child=DataSourceExec +12)----------------------DataSourceExec: 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], file_type=csv, 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 p_container@3 IN ([Literal { value: Utf8View("SM CASE") }, Literal { value: Utf8View("SM BOX") }, Literal { value: Utf8View("SM PACK") }, Literal { value: Utf8View("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG") }, Literal { value: Utf8View("MED BOX") }, Literal { value: Utf8View("MED PKG") }, Literal { value: Utf8View("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE") }, Literal { value: Utf8View("LG BOX") }, Literal { value: Utf8View("LG PACK") }, Literal { value: Utf8View("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 +17)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------YieldStreamExec child=DataSourceExec +19)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index b2e0fb0cd1cc..18cd8ab1e3b7 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -126,55 +126,64 @@ physical_plan 25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 26)--------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] 27)----------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false -29)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -31)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -32)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -34)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -36)--------------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -38)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -39)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -40)------------------CoalesceBatchesExec: target_batch_size=8192 -41)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -42)----------------------CoalesceBatchesExec: target_batch_size=8192 -43)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -44)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false -46)----------CoalesceBatchesExec: target_batch_size=8192 -47)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -48)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -49)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -50)------------------CoalesceBatchesExec: target_batch_size=8192 -51)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -52)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -53)------------------------CoalesceBatchesExec: target_batch_size=8192 -54)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -55)----------------------------CoalesceBatchesExec: target_batch_size=8192 -56)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -57)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -59)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -60)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -61)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -62)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -63)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -64)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -65)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -66)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -67)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -68)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -69)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -70)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -71)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -72)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -73)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -74)----------------------------CoalesceBatchesExec: target_batch_size=8192 -75)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -76)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -77)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -78)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -79)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +28)------------------------------------------------------YieldStreamExec child=DataSourceExec +29)--------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false +30)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +32)------------------------------------------------YieldStreamExec child=DataSourceExec +33)--------------------------------------------------DataSourceExec: 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], file_type=csv, 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)------------------------------------------YieldStreamExec child=DataSourceExec +38)--------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +39)--------------------------CoalesceBatchesExec: target_batch_size=8192 +40)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +41)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +42)--------------------------------YieldStreamExec child=DataSourceExec +43)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false +44)------------------CoalesceBatchesExec: target_batch_size=8192 +45)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +46)----------------------CoalesceBatchesExec: target_batch_size=8192 +47)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +48)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +49)----------------------------YieldStreamExec child=DataSourceExec +50)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +51)----------CoalesceBatchesExec: target_batch_size=8192 +52)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +53)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +54)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +55)------------------CoalesceBatchesExec: target_batch_size=8192 +56)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +57)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +58)------------------------CoalesceBatchesExec: target_batch_size=8192 +59)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +60)----------------------------CoalesceBatchesExec: target_batch_size=8192 +61)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +62)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +63)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +64)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +65)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +66)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +67)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +68)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +69)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +70)------------------------------------------------YieldStreamExec child=DataSourceExec +71)--------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +72)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +73)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +74)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +75)--------------------------------------------------YieldStreamExec child=DataSourceExec +76)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +77)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +78)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +79)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +80)------------------------------------------YieldStreamExec child=DataSourceExec +81)--------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false +82)----------------------------CoalesceBatchesExec: target_batch_size=8192 +83)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +84)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +85)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +86)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +87)--------------------------------------YieldStreamExec child=DataSourceExec +88)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 0b994de411ea..8ff8388687fb 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -92,35 +92,40 @@ physical_plan 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)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=csv, has_header=false -13)----------------CoalesceBatchesExec: target_batch_size=8192 -14)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -15)--------------------CoalesceBatchesExec: target_batch_size=8192 -16)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] -17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -19)--------CoalesceBatchesExec: target_batch_size=8192 -20)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -21)------------CoalesceBatchesExec: target_batch_size=8192 -22)--------------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] -23)----------------CoalesceBatchesExec: target_batch_size=8192 -24)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -25)--------------------CoalesceBatchesExec: target_batch_size=8192 -26)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -27)------------------------CoalesceBatchesExec: target_batch_size=8192 -28)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -29)----------------------------DataSourceExec: 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], file_type=csv, has_header=false -30)------------------------CoalesceBatchesExec: target_batch_size=8192 -31)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -32)----------------------------CoalesceBatchesExec: target_batch_size=8192 -33)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] -34)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false -36)----------------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] -37)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -38)--------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -40)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -41)--------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] -43)------------------------------DataSourceExec: 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], file_type=csv, has_header=false +12)----------------------YieldStreamExec child=DataSourceExec +13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=csv, has_header=false +14)----------------CoalesceBatchesExec: target_batch_size=8192 +15)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +16)--------------------CoalesceBatchesExec: target_batch_size=8192 +17)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] +18)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)--------------------------YieldStreamExec child=DataSourceExec +20)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +21)--------CoalesceBatchesExec: target_batch_size=8192 +22)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +23)------------CoalesceBatchesExec: target_batch_size=8192 +24)--------------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] +25)----------------CoalesceBatchesExec: target_batch_size=8192 +26)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +27)--------------------CoalesceBatchesExec: target_batch_size=8192 +28)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +29)------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +31)----------------------------YieldStreamExec child=DataSourceExec +32)------------------------------DataSourceExec: 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], file_type=csv, has_header=false +33)------------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +35)----------------------------CoalesceBatchesExec: target_batch_size=8192 +36)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] +37)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +38)----------------------------------YieldStreamExec child=DataSourceExec +39)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false +40)----------------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] +41)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +42)--------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +44)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +45)--------------------------CoalesceBatchesExec: target_batch_size=8192 +46)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] +47)------------------------------YieldStreamExec child=DataSourceExec +48)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index e52171524007..fe7ae6f6f03b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -116,28 +116,34 @@ physical_plan 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)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=csv, has_header=false -28)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -29)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -32)------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -33)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] -37)----------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -38)------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -40)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] -42)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -43)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -44)----------------------CoalesceBatchesExec: target_batch_size=8192 -45)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -46)--------------------------DataSourceExec: 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], file_type=csv, has_header=false -47)------------------CoalesceBatchesExec: target_batch_size=8192 -48)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -49)----------------------CoalesceBatchesExec: target_batch_size=8192 -50)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -51)--------------------------DataSourceExec: 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], file_type=csv, has_header=false +27)----------------------------------------------------YieldStreamExec child=DataSourceExec +28)------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=csv, has_header=false +29)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +31)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +33)------------------------------------------------------YieldStreamExec child=DataSourceExec +34)--------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +35)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +37)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] +39)----------------------------------------------YieldStreamExec child=DataSourceExec +40)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +41)------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +43)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] +45)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +46)----------------------------------------YieldStreamExec child=DataSourceExec +47)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +48)----------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +50)--------------------------YieldStreamExec child=DataSourceExec +51)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +52)------------------CoalesceBatchesExec: target_batch_size=8192 +53)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +54)----------------------CoalesceBatchesExec: target_batch_size=8192 +55)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +56)--------------------------YieldStreamExec child=DataSourceExec +57)----------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index 828bf967d8f4..71b1f0554da1 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -92,14 +92,17 @@ physical_plan 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 17)--------------------------------FilterExec: substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13") }, Literal { value: Utf8View("31") }, Literal { value: Utf8View("23") }, Literal { value: Utf8View("29") }, Literal { value: Utf8View("30") }, Literal { value: Utf8View("18") }, Literal { value: Utf8View("17") }]) 18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false -20)--------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -22)------------------------------DataSourceExec: 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], file_type=csv, has_header=false -23)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] -24)----------------------CoalescePartitionsExec -25)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -26)--------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13") }, Literal { value: Utf8View("31") }, Literal { value: Utf8View("23") }, Literal { value: Utf8View("29") }, Literal { value: Utf8View("30") }, Literal { value: Utf8View("18") }, Literal { value: Utf8View("17") }]), projection=[c_acctbal@1] -28)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false +19)------------------------------------YieldStreamExec child=DataSourceExec +20)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false +21)--------------------------CoalesceBatchesExec: target_batch_size=8192 +22)----------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 +23)------------------------------YieldStreamExec child=DataSourceExec +24)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false +25)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] +26)----------------------CoalescePartitionsExec +27)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] +28)--------------------------CoalesceBatchesExec: target_batch_size=8192 +29)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13") }, Literal { value: Utf8View("31") }, Literal { value: Utf8View("23") }, Literal { value: Utf8View("29") }, Literal { value: Utf8View("30") }, Literal { value: Utf8View("18") }, Literal { value: Utf8View("17") }]), projection=[c_acctbal@1] +30)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +31)--------------------------------YieldStreamExec child=DataSourceExec +32)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index d982ec32e954..99aa0cc70e9e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -76,14 +76,17 @@ physical_plan 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 17)--------------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] 18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], file_type=csv, has_header=false -20)--------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -22)------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 -24)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false -25)------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -27)----------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -29)--------------------------DataSourceExec: 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], file_type=csv, has_header=false +19)------------------------------------YieldStreamExec child=DataSourceExec +20)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], file_type=csv, has_header=false +21)--------------------------CoalesceBatchesExec: target_batch_size=8192 +22)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +23)------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 +25)----------------------------------YieldStreamExec child=DataSourceExec +26)------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +27)------------------CoalesceBatchesExec: target_batch_size=8192 +28)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +29)----------------------CoalesceBatchesExec: target_batch_size=8192 +30)------------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +31)--------------------------YieldStreamExec child=DataSourceExec +32)----------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index f7de3cd3c967..d8194c838409 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -66,9 +66,11 @@ physical_plan 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] -14)--------------------------DataSourceExec: 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], file_type=csv, has_header=false -15)------------------CoalesceBatchesExec: target_batch_size=8192 -16)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -17)----------------------CoalesceBatchesExec: target_batch_size=8192 -18)------------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] -19)--------------------------DataSourceExec: 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], file_type=csv, has_header=false +14)--------------------------YieldStreamExec child=DataSourceExec +15)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +16)------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +18)----------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] +20)--------------------------YieldStreamExec child=DataSourceExec +21)----------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index 15636056b871..54c1068e2697 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -95,26 +95,32 @@ physical_plan 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)--------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -32)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -34)----------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -37)----------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -38)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 -40)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -41)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -42)--------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -46)------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -48)----------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] -50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -51)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +29)--------------------------------------------------------YieldStreamExec child=DataSourceExec +30)----------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false +31)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +33)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +35)----------------------------------------------------------YieldStreamExec child=DataSourceExec +36)------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +37)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +39)----------------------------------------------YieldStreamExec child=DataSourceExec +40)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +41)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 +43)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +44)----------------------------------------YieldStreamExec child=DataSourceExec +45)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +46)--------------------------CoalesceBatchesExec: target_batch_size=8192 +47)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +48)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +49)--------------------------------YieldStreamExec child=DataSourceExec +50)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false +51)------------------CoalesceBatchesExec: target_batch_size=8192 +52)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +53)----------------------CoalesceBatchesExec: target_batch_size=8192 +54)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +55)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +56)----------------------------YieldStreamExec child=DataSourceExec +57)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part index b1e5d2869a8c..1d026866ef91 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part @@ -40,4 +40,5 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] -07)------------DataSourceExec: 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], file_type=csv, has_header=false +07)------------YieldStreamExec child=DataSourceExec +08)--------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index 291d56e43f2d..8b0c733f701c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -113,28 +113,34 @@ physical_plan 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)----------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, 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 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 -35)------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -38)------------------------------------------------DataSourceExec: 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], file_type=csv, 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)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, 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)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, 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)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +30)----------------------------------------------------------YieldStreamExec child=DataSourceExec +31)------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +32)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +34)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)----------------------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 +36)------------------------------------------------------------YieldStreamExec child=DataSourceExec +37)--------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +38)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +40)------------------------------------------------YieldStreamExec child=DataSourceExec +41)--------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +42)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +44)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +45)------------------------------------------YieldStreamExec child=DataSourceExec +46)--------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false +47)----------------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +49)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +50)----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +51)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +52)--------------------------------------YieldStreamExec child=DataSourceExec +53)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +54)--------------------CoalesceBatchesExec: target_batch_size=8192 +55)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +56)------------------------CoalesceBatchesExec: target_batch_size=8192 +57)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +58)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +59)------------------------------YieldStreamExec child=DataSourceExec +60)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 50171c528db6..80a53ad3dbff 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -129,34 +129,42 @@ physical_plan 37)------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 38)--------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] 39)----------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -40)------------------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false -41)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -43)------------------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -44)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -45)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -46)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -48)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -50)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -51)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -52)------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -53)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -54)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -55)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -56)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -57)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -59)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -60)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -61)----------------------------CoalesceBatchesExec: target_batch_size=8192 -62)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -63)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -64)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -65)--------------------CoalesceBatchesExec: target_batch_size=8192 -66)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -67)------------------------CoalesceBatchesExec: target_batch_size=8192 -68)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] -69)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -70)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +40)------------------------------------------------------------------------------YieldStreamExec child=DataSourceExec +41)--------------------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false +42)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +44)------------------------------------------------------------------------YieldStreamExec child=DataSourceExec +45)--------------------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +46)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +47)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +48)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +49)------------------------------------------------------------------YieldStreamExec child=DataSourceExec +50)--------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +51)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +52)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +53)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +54)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +55)------------------------------------------------------------YieldStreamExec child=DataSourceExec +56)--------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +57)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +59)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +60)--------------------------------------------------YieldStreamExec child=DataSourceExec +61)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, 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)------------------------------------------YieldStreamExec child=DataSourceExec +66)--------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false +67)----------------------------CoalesceBatchesExec: target_batch_size=8192 +68)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +69)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +70)----------------------------------YieldStreamExec child=DataSourceExec +71)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +72)--------------------CoalesceBatchesExec: target_batch_size=8192 +73)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +74)------------------------CoalesceBatchesExec: target_batch_size=8192 +75)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] +76)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +77)------------------------------YieldStreamExec child=DataSourceExec +78)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 3b31c1bc2e8e..4290d93f57dc 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -106,21 +106,27 @@ physical_plan 29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 30)----------------------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] 31)------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)--------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false -33)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -35)--------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -38)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -39)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -40)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -42)----------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -43)----------------------------CoalesceBatchesExec: target_batch_size=8192 -44)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -45)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false -46)--------------------CoalesceBatchesExec: target_batch_size=8192 -47)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -48)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -49)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +32)--------------------------------------------------------------YieldStreamExec child=DataSourceExec +33)----------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false +34)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +36)--------------------------------------------------------YieldStreamExec child=DataSourceExec +37)----------------------------------------------------------DataSourceExec: 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], file_type=csv, 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)--------------------------------------------------YieldStreamExec child=DataSourceExec +42)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +43)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +45)----------------------------------------YieldStreamExec child=DataSourceExec +46)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +47)----------------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +49)--------------------------------YieldStreamExec child=DataSourceExec +50)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false +51)--------------------CoalesceBatchesExec: target_batch_size=8192 +52)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +53)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +54)--------------------------YieldStreamExec child=DataSourceExec +55)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false From 2887f872c2b0a49314f8cffd5eea6a518ba9b8dd Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 4 Jun 2025 21:59:13 +0800 Subject: [PATCH 32/74] Add flag for yield insert and disable default --- datafusion-examples/examples/planner_api.rs | 18 +- datafusion/common/src/config.rs | 6 + .../core/tests/execution/infinite_cancel.rs | 9 +- .../enforce_distribution.rs | 8 +- .../src/wrap_leaves_cancellation.rs | 14 +- .../test_files/agg_func_substitute.slt | 9 +- .../sqllogictest/test_files/aggregate.slt | 51 ++-- .../test_files/aggregates_topk.slt | 21 +- datafusion/sqllogictest/test_files/alias.slt | 12 +- datafusion/sqllogictest/test_files/array.slt | 18 +- datafusion/sqllogictest/test_files/avro.slt | 3 +- .../test_files/count_star_rule.slt | 9 +- .../test_files/create_external_table.slt | 3 +- .../sqllogictest/test_files/csv_files.slt | 3 +- datafusion/sqllogictest/test_files/cte.slt | 32 +-- .../sqllogictest/test_files/distinct_on.slt | 3 +- .../sqllogictest/test_files/explain.slt | 3 +- .../sqllogictest/test_files/explain_tree.slt | 210 +++++----------- datafusion/sqllogictest/test_files/expr.slt | 3 +- .../sqllogictest/test_files/group_by.slt | 150 ++++------- .../test_files/information_schema.slt | 2 + datafusion/sqllogictest/test_files/insert.slt | 12 +- .../test_files/insert_to_external.slt | 12 +- .../sqllogictest/test_files/join.slt.part | 19 +- datafusion/sqllogictest/test_files/joins.slt | 238 +++++++----------- datafusion/sqllogictest/test_files/json.slt | 3 +- datafusion/sqllogictest/test_files/limit.slt | 36 +-- .../test_files/monotonic_projection_test.slt | 9 +- datafusion/sqllogictest/test_files/order.slt | 54 ++-- .../sqllogictest/test_files/parquet.slt | 6 +- .../test_files/parquet_filter_pushdown.slt | 12 +- .../test_files/parquet_sorted_statistics.slt | 3 +- .../sqllogictest/test_files/predicates.slt | 29 +-- .../sqllogictest/test_files/references.slt | 3 +- .../sqllogictest/test_files/repartition.slt | 6 +- .../test_files/repartition_scan.slt | 3 +- datafusion/sqllogictest/test_files/select.slt | 15 +- .../test_files/sort_merge_join.slt | 8 +- .../sqllogictest/test_files/subquery.slt | 28 +-- .../sqllogictest/test_files/subquery_sort.slt | 12 +- datafusion/sqllogictest/test_files/topk.slt | 30 +-- datafusion/sqllogictest/test_files/union.slt | 138 +++++----- datafusion/sqllogictest/test_files/window.slt | 186 +++++--------- 43 files changed, 547 insertions(+), 902 deletions(-) diff --git a/datafusion-examples/examples/planner_api.rs b/datafusion-examples/examples/planner_api.rs index 051d9a2d98e2..55aec7b0108a 100644 --- a/datafusion-examples/examples/planner_api.rs +++ b/datafusion-examples/examples/planner_api.rs @@ -18,6 +18,7 @@ use datafusion::error::Result; use datafusion::logical_expr::LogicalPlan; use datafusion::physical_plan::displayable; +use datafusion::physical_planner::DefaultPhysicalPlanner; use datafusion::prelude::*; /// This example demonstrates the process of converting logical plan @@ -116,21 +117,18 @@ async fn to_physical_plan_step_by_step_demo( displayable(physical_plan.as_ref()).indent(false) ); - // todo, the following code is commented out because it is actually not a right way to optimize based a already optimized plan. - // And for YieldStreamExec, it shows error when optimizing it again based on the already optimized plan which already has YieldStreamExec. - // Call the physical optimizer with an existing physical plan (in this // case the plan is already optimized, but an unoptimized plan would // typically be used in this context) // Note that this is not part of the trait but a public method // on DefaultPhysicalPlanner. Not all planners will provide this feature. - // let planner = DefaultPhysicalPlanner::default(); - // let physical_plan = - // planner.optimize_physical_plan(physical_plan, &ctx.state(), |_, _| {})?; - // println!( - // "Optimized physical plan:\n\n{}\n\n", - // displayable(physical_plan.as_ref()).indent(false) - // ); + let planner = DefaultPhysicalPlanner::default(); + let physical_plan = + planner.optimize_physical_plan(physical_plan, &ctx.state(), |_, _| {})?; + println!( + "Optimized physical plan:\n\n{}\n\n", + displayable(physical_plan.as_ref()).indent(false) + ); Ok(()) } diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 726015d17149..bc449dc93271 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -722,6 +722,12 @@ config_namespace! { /// then the output will be coerced to a non-view. /// Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. pub expand_views_at_output: bool, default = false + + /// When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline + /// that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to + /// other tasks while waiting. + /// Default: false (disabled). + pub enable_add_yield_for_pipeline_break: bool, default = false } } diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index 57ff42aca772..8afb6a98d610 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -172,7 +172,9 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { )?); // 3) optimize the plan with WrapLeaves to auto-insert Yield - let optimized = WrapLeaves::new().optimize(aggr.clone(), &ConfigOptions::new())?; + let mut config = ConfigOptions::new(); + config.optimizer.enable_add_yield_for_pipeline_break = true; + let optimized = WrapLeaves::new().optimize(aggr.clone(), &config)?; // 4) get the stream let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; @@ -223,8 +225,9 @@ async fn test_infinite_sort_cancel() -> Result<(), Box> { let sort_exec = Arc::new(SortExec::new(lex_ordering, inf.clone())); // 4) optimize the plan with WrapLeaves to auto-insert Yield - let optimized = - WrapLeaves::new().optimize(sort_exec.clone(), &ConfigOptions::new())?; + let mut config = ConfigOptions::new(); + config.optimizer.enable_add_yield_for_pipeline_break = true; + let optimized = WrapLeaves::new().optimize(sort_exec.clone(), &config)?; // 5) get the stream let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 21e40d6ede61..638156838798 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3523,13 +3523,19 @@ async fn test_distribute_sort_parquet() -> Result<()> { /// Ensures that `DataSourceExec` has been repartitioned into `target_partitions` memtable groups #[tokio::test] async fn test_distribute_sort_memtable() -> Result<()> { - let test_config: TestConfig = + let mut test_config: TestConfig = TestConfig::default().with_prefer_repartition_file_scans(1000); assert!( test_config.config.optimizer.repartition_file_scans, "should enable scans to be repartitioned" ); + // Enable add yield for pipeline break testing. + test_config + .config + .optimizer + .enable_add_yield_for_pipeline_break = true; + let mem_table = create_memtable()?; let session_config = SessionConfig::new() .with_repartition_file_min_size(1000) diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index 63311b1abd7c..d8829e3987c6 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -98,12 +98,16 @@ impl PhysicalOptimizerRule for WrapLeaves { fn optimize( &self, plan: Arc, - _config: &ConfigOptions, + config: &ConfigOptions, ) -> Result> { - // We run a top‐level transform_down: for every node, call wrap_leaves_of_pipeline_breakers. - // If a node is a pipeline breaker, we then wrap all of its leaf children in YieldStreamExec. - plan.transform_down(Self::wrap_leaves_of_pipeline_breakers) - .map(|t| t.data) + if config.optimizer.enable_add_yield_for_pipeline_break { + // We run a top‐level transform_down: for every node, call wrap_leaves_of_pipeline_breakers. + // If a node is a pipeline breaker, we then wrap all of its leaf children in YieldStreamExec. + plan.transform_down(Self::wrap_leaves_of_pipeline_breakers) + .map(|t| t.data) + } else { + Ok(plan) + } } fn schema_check(&self) -> bool { diff --git a/datafusion/sqllogictest/test_files/agg_func_substitute.slt b/datafusion/sqllogictest/test_files/agg_func_substitute.slt index d512a77e543d..9aeaaacb1071 100644 --- a/datafusion/sqllogictest/test_files/agg_func_substitute.slt +++ b/datafusion/sqllogictest/test_files/agg_func_substitute.slt @@ -50,8 +50,7 @@ physical_plan 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)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: 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]], file_type=csv, has_header=true +08)--------------DataSourceExec: 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]], file_type=csv, has_header=true query TT @@ -71,8 +70,7 @@ physical_plan 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)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: 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]], file_type=csv, has_header=true +08)--------------DataSourceExec: 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]], file_type=csv, has_header=true query TT EXPLAIN SELECT a, ARRAY_AGG(c ORDER BY c)[1 + 100] as result @@ -91,8 +89,7 @@ physical_plan 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)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: 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]], file_type=csv, has_header=true +08)--------------DataSourceExec: 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]], file_type=csv, 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 ee8982907d93..52b1e1c22fdf 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -250,8 +250,7 @@ physical_plan 03)----AggregateExec: mode=Partial, gby=[], aggr=[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] 04)------SortExec: expr=[c2@1 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[true] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true # test array_agg_order with list data type statement ok @@ -1131,8 +1130,7 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=4 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; @@ -5296,8 +5294,7 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true # @@ -5322,8 +5319,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], file_type=csv, has_header=true query I SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 order by c3 limit 5; @@ -5347,8 +5343,7 @@ physical_plan 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)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query II SELECT c2, c3 FROM aggregate_test_100 group by c2, c3 order by c2, c3 limit 5 offset 4; @@ -5383,8 +5378,7 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------------------YieldStreamExec child=DataSourceExec -14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true +13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query I SELECT DISTINCT c3 FROM aggregate_test_100 WHERE c3 between 10 and 20 group by c3 order by c3 limit 4; @@ -5410,8 +5404,7 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, 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 @@ -5435,8 +5428,7 @@ physical_plan 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)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query II SELECT DISTINCT c3, c2 FROM aggregate_test_100 group by c3, c2 order by c3, c2 limit 3 offset 10; @@ -5460,8 +5452,7 @@ physical_plan 04)------CoalescePartitionsExec 05)--------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=[] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query II SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; @@ -5488,8 +5479,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], file_type=csv, has_header=true statement ok set datafusion.optimizer.enable_distinct_aggregation_soft_limit = true; @@ -6318,8 +6308,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[last_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 ASC NULLS LAST]] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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]], file_type=csv, has_header=true +05)--------DataSourceExec: 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]], file_type=csv, has_header=true # test last to first query TT @@ -6333,8 +6322,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[first_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 DESC NULLS FIRST]] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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]], file_type=csv, has_header=true +05)--------DataSourceExec: 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]], file_type=csv, has_header=true # test building plan with aggreagte sum @@ -6407,8 +6395,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query error @@ -6583,8 +6570,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(aggregate_test_100.c5)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], file_type=csv, has_header=true statement count 0 drop table aggregate_test_100; @@ -6607,8 +6593,7 @@ logical_plan 02)--TableScan: t projection=[] physical_plan 01)AggregateExec: mode=Single, gby=[], aggr=[count(NULL)] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement count 0 drop table t; @@ -6682,8 +6667,7 @@ logical_plan 02)--TableScan: t projection=[] physical_plan 01)AggregateExec: mode=Single, gby=[], aggr=[count(Int64(1)), count(Int64(2))] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] query II select count(1), count() from t; @@ -6743,8 +6727,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 * count(Int64(2))@1 as count(Int64(1)) * count(Int64(2))] 02)--AggregateExec: mode=Single, gby=[], aggr=[count(Int64(1)), count(Int64(2))] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement count 0 drop table t; diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index 96a9da7d1665..cc1693843848 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -50,8 +50,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TI select * from (select trace_id, MAX(timestamp) max_ts from traces t group by trace_id) where trace_id != 'b' order by max_ts desc limit 3; @@ -115,8 +114,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: 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; @@ -133,8 +131,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: 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; @@ -151,8 +148,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: 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; @@ -169,8 +165,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: 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; @@ -244,8 +239,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces_utf8view.timestamp)], lim=[4] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] # Also add LargeUtf8 to test PR https://github.com/apache/datafusion/pull/15152 @@ -273,8 +267,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces_largeutf8.timestamp)], lim=[4] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/alias.slt b/datafusion/sqllogictest/test_files/alias.slt index 780d12d4ffdf..5339179db4c4 100644 --- a/datafusion/sqllogictest/test_files/alias.slt +++ b/datafusion/sqllogictest/test_files/alias.slt @@ -34,10 +34,8 @@ logical_plan 04)----TableScan: t2 projection=[age] physical_plan 01)CrossJoinExec -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)--YieldStreamExec child=DataSourceExec -05)----DataSourceExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[0] query TT explain select * from ((select id from t1) cross join (select age from t2)) as f(c1, c2); @@ -51,10 +49,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as c1, age@1 as c2] 02)--CrossJoinExec -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table t1; diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 756dbb9d53c1..d89ba600d7a6 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -6034,8 +6034,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec child=LazyMemoryExec -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6064,8 +6063,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec child=LazyMemoryExec -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6094,8 +6092,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec child=LazyMemoryExec -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # FIXME: due to rewrite below not working, this is _extremely_ slow to evaluate # query I @@ -6126,8 +6123,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: array_has([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c], substr(md5(CAST(value@0 AS Utf8)), 1, 32)) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec child=LazyMemoryExec -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6156,8 +6152,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec child=LazyMemoryExec -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6188,8 +6183,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IS NOT NULL OR NULL 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec child=LazyMemoryExec -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # any operator query ? diff --git a/datafusion/sqllogictest/test_files/avro.slt b/datafusion/sqllogictest/test_files/avro.slt index 4da87511dced..4573af1d59b1 100644 --- a/datafusion/sqllogictest/test_files/avro.slt +++ b/datafusion/sqllogictest/test_files/avro.slt @@ -256,8 +256,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]}, file_type=avro +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]}, file_type=avro # test column projection order from avro file query ITII diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index 8da0eb14c1c2..d38d3490fed4 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -53,8 +53,7 @@ physical_plan 04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 0; @@ -73,8 +72,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 1; @@ -92,8 +90,7 @@ physical_plan 01)ProjectionExec: expr=[a@0 as a, count(Int64(1)) PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as count_a] 02)--WindowAggExec: wdw=[count(Int64(1)) PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(Int64(1)) PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, 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 }] 03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT a, COUNT() OVER (PARTITION BY a) AS count_a FROM t1 ORDER BY a; diff --git a/datafusion/sqllogictest/test_files/create_external_table.slt b/datafusion/sqllogictest/test_files/create_external_table.slt index 8075ff1b5884..03cb5edb5fcc 100644 --- a/datafusion/sqllogictest/test_files/create_external_table.slt +++ b/datafusion/sqllogictest/test_files/create_external_table.slt @@ -264,8 +264,7 @@ logical_plan 02)--TableScan: t projection=[id] physical_plan 01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet statement ok DROP TABLE t; diff --git a/datafusion/sqllogictest/test_files/csv_files.slt b/datafusion/sqllogictest/test_files/csv_files.slt index 94700c8e7811..5a7fa309dbfa 100644 --- a/datafusion/sqllogictest/test_files/csv_files.slt +++ b/datafusion/sqllogictest/test_files/csv_files.slt @@ -165,8 +165,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [int_col@0 ASC NULLS LAST] 02)--SortExec: expr=[int_col@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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], file_type=csv, has_header=false +03)----DataSourceExec: 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], file_type=csv, has_header=false # ensure that correct quote character is used when writing to csv diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 6a07eb7d0a1a..32320a06f4fb 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -160,15 +160,13 @@ logical_plan physical_plan 01)SortExec: expr=[time@0 ASC NULLS LAST, name@1 ASC NULLS LAST, account_balance@2 ASC NULLS LAST], preserve_partitioning=[false] 02)--RecursiveQueryExec: name=balances, is_distinct=false -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], file_type=csv, has_header=true -05)----CoalescePartitionsExec -06)------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] -07)--------CoalesceBatchesExec: target_batch_size=2 -08)----------FilterExec: time@0 < 10 -09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)--------------YieldStreamExec child=WorkTableExec -11)----------------WorkTableExec: name=balances +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], file_type=csv, 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 @@ -740,11 +738,9 @@ physical_plan 07)--------CoalesceBatchesExec: target_batch_size=8182 08)----------FilterExec: val@0 < 2 09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)--------------YieldStreamExec child=WorkTableExec -11)----------------WorkTableExec: name=recursive_cte -12)------ProjectionExec: expr=[2 as val] -13)--------YieldStreamExec child=PlaceholderRowExec -14)----------PlaceholderRowExec +10)--------------WorkTableExec: name=recursive_cte +11)------ProjectionExec: expr=[2 as val] +12)--------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/9794 # Non-recursive term and recursive term have different types @@ -938,11 +934,9 @@ logical_plan 05)------EmptyRelation physical_plan 01)CrossJoinExec -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)--ProjectionExec: expr=[1 as Int64(1)] -05)----YieldStreamExec child=PlaceholderRowExec -06)------PlaceholderRowExec +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--ProjectionExec: expr=[1 as Int64(1)] +04)----PlaceholderRowExec statement count 0 drop table j1; diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 167c4dd9cf96..b4a491619e89 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -102,8 +102,7 @@ physical_plan 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) 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]] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, 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 010fdae34546..8c9054d56c14 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -165,8 +165,7 @@ logical_plan physical_plan 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], preserve_partitioning=[false] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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], file_type=csv, has_header=true +03)----DataSourceExec: 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], file_type=csv, has_header=true # test EXPLAIN VERBOSE query TT diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index c847e67bdb91..15bf61576571 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -243,17 +243,11 @@ physical_plan 43)│ RoundRobinBatch(4) │ 44)└─────────────┬─────────────┘ 45)┌─────────────┴─────────────┐ -46)│ YieldStreamExec │ +46)│ DataSourceExec │ 47)│ -------------------- │ -48)│ YieldStreamExec child: │ -49)│ DataSourceExec │ -50)└─────────────┬─────────────┘ -51)┌─────────────┴─────────────┐ -52)│ DataSourceExec │ -53)│ -------------------- │ -54)│ files: 1 │ -55)│ format: csv │ -56)└───────────────────────────┘ +48)│ files: 1 │ +49)│ format: csv │ +50)└───────────────────────────┘ # Limit @@ -773,17 +767,11 @@ physical_plan 19)│ FOLLOWING │ 20)└─────────────┬─────────────┘ 21)┌─────────────┴─────────────┐ -22)│ YieldStreamExec │ +22)│ DataSourceExec │ 23)│ -------------------- │ -24)│ YieldStreamExec child: │ -25)│ DataSourceExec │ -26)└─────────────┬─────────────┘ -27)┌─────────────┴─────────────┐ -28)│ DataSourceExec │ -29)│ -------------------- │ -30)│ files: 1 │ -31)│ format: csv │ -32)└───────────────────────────┘ +24)│ files: 1 │ +25)│ format: csv │ +26)└───────────────────────────┘ # Query with bounded window agg. query TT @@ -826,19 +814,13 @@ physical_plan 31)│ v1: value │ 32)└─────────────┬─────────────┘ 33)┌─────────────┴─────────────┐ -34)│ YieldStreamExec │ +34)│ LazyMemoryExec │ 35)│ -------------------- │ -36)│ YieldStreamExec child: │ -37)│ LazyMemoryExec │ -38)└─────────────┬─────────────┘ -39)┌─────────────┴─────────────┐ -40)│ LazyMemoryExec │ -41)│ -------------------- │ -42)│ batch_generators: │ -43)│ generate_series: start=1, │ -44)│ end=1000, batch_size │ -45)│ =8192 │ -46)└───────────────────────────┘ +36)│ batch_generators: │ +37)│ generate_series: start=1, │ +38)│ end=1000, batch_size │ +39)│ =8192 │ +40)└───────────────────────────┘ query TT explain select @@ -878,17 +860,11 @@ physical_plan 29)│ FOLLOWING │ 30)└─────────────┬─────────────┘ 31)┌─────────────┴─────────────┐ -32)│ YieldStreamExec │ +32)│ DataSourceExec │ 33)│ -------------------- │ -34)│ YieldStreamExec child: │ -35)│ DataSourceExec │ -36)└─────────────┬─────────────┘ -37)┌─────────────┴─────────────┐ -38)│ DataSourceExec │ -39)│ -------------------- │ -40)│ files: 1 │ -41)│ format: csv │ -42)└───────────────────────────┘ +34)│ files: 1 │ +35)│ format: csv │ +36)└───────────────────────────┘ # Query for sort. query TT @@ -901,17 +877,11 @@ physical_plan 04)│string_col@1 ASC NULLS LAST│ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ YieldStreamExec │ +07)│ DataSourceExec │ 08)│ -------------------- │ -09)│ YieldStreamExec child: │ -10)│ DataSourceExec │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ DataSourceExec │ -14)│ -------------------- │ -15)│ files: 1 │ -16)│ format: csv │ -17)└───────────────────────────┘ +09)│ files: 1 │ +10)│ format: csv │ +11)└───────────────────────────┘ # Query for sort with limit. query TT @@ -926,17 +896,11 @@ physical_plan 06)│string_col@1 ASC NULLS LAST│ 07)└─────────────┬─────────────┘ 08)┌─────────────┴─────────────┐ -09)│ YieldStreamExec │ +09)│ DataSourceExec │ 10)│ -------------------- │ -11)│ YieldStreamExec child: │ -12)│ DataSourceExec │ -13)└─────────────┬─────────────┘ -14)┌─────────────┴─────────────┐ -15)│ DataSourceExec │ -16)│ -------------------- │ -17)│ files: 1 │ -18)│ format: csv │ -19)└───────────────────────────┘ +11)│ files: 1 │ +12)│ format: csv │ +13)└───────────────────────────┘ # Query with projection on csv query TT @@ -1041,17 +1005,11 @@ physical_plan 61)│ int_col@0 DESC │ 62)└─────────────┬─────────────┘ 63)┌─────────────┴─────────────┐ -64)│ YieldStreamExec │ +64)│ DataSourceExec │ 65)│ -------------------- │ -66)│ YieldStreamExec child: │ -67)│ DataSourceExec │ -68)└─────────────┬─────────────┘ -69)┌─────────────┴─────────────┐ -70)│ DataSourceExec │ -71)│ -------------------- │ -72)│ files: 1 │ -73)│ format: csv │ -74)└───────────────────────────┘ +66)│ files: 1 │ +67)│ format: csv │ +68)└───────────────────────────┘ # Query with projection on parquet query TT @@ -1375,17 +1333,17 @@ physical_plan 04)│ join_type: LeftSemi │ │ 05)└─────────────┬─────────────┘ │ 06)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -07)│ YieldStreamExec ││ ProjectionExec │ +07)│ DataSourceExec ││ ProjectionExec │ 08)│ -------------------- ││ │ -09)│ YieldStreamExec child: ││ │ -10)│ DataSourceExec ││ │ -11)└─────────────┬─────────────┘└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -13)│ DataSourceExec ││ AggregateExec │ -14)│ -------------------- ││ -------------------- │ -15)│ files: 1 ││ aggr: count(1) │ -16)│ format: csv ││ mode: Final │ -17)└───────────────────────────┘└─────────────┬─────────────┘ +09)│ files: 1 ││ │ +10)│ format: csv ││ │ +11)└───────────────────────────┘└─────────────┬─────────────┘ +12)-----------------------------┌─────────────┴─────────────┐ +13)-----------------------------│ AggregateExec │ +14)-----------------------------│ -------------------- │ +15)-----------------------------│ aggr: count(1) │ +16)-----------------------------│ mode: Final │ +17)-----------------------------└─────────────┬─────────────┘ 18)-----------------------------┌─────────────┴─────────────┐ 19)-----------------------------│ CoalescePartitionsExec │ 20)-----------------------------└─────────────┬─────────────┘ @@ -1405,17 +1363,11 @@ physical_plan 34)-----------------------------│ RoundRobinBatch(4) │ 35)-----------------------------└─────────────┬─────────────┘ 36)-----------------------------┌─────────────┴─────────────┐ -37)-----------------------------│ YieldStreamExec │ +37)-----------------------------│ DataSourceExec │ 38)-----------------------------│ -------------------- │ -39)-----------------------------│ YieldStreamExec child: │ -40)-----------------------------│ DataSourceExec │ -41)-----------------------------└─────────────┬─────────────┘ -42)-----------------------------┌─────────────┴─────────────┐ -43)-----------------------------│ DataSourceExec │ -44)-----------------------------│ -------------------- │ -45)-----------------------------│ files: 1 │ -46)-----------------------------│ format: parquet │ -47)-----------------------------└───────────────────────────┘ +39)-----------------------------│ files: 1 │ +40)-----------------------------│ format: parquet │ +41)-----------------------------└───────────────────────────┘ # Query with cross join. query TT @@ -1426,26 +1378,20 @@ physical_plan 02)│ CrossJoinExec ├──────────────┐ 03)└─────────────┬─────────────┘ │ 04)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -05)│ YieldStreamExec ││ RepartitionExec │ +05)│ DataSourceExec ││ RepartitionExec │ 06)│ -------------------- ││ -------------------- │ -07)│ YieldStreamExec child: ││ partition_count(in->out): │ -08)│ DataSourceExec ││ 1 -> 4 │ +07)│ files: 1 ││ partition_count(in->out): │ +08)│ format: csv ││ 1 -> 4 │ 09)│ ││ │ 10)│ ││ partitioning_scheme: │ 11)│ ││ RoundRobinBatch(4) │ -12)└─────────────┬─────────────┘└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -14)│ DataSourceExec ││ YieldStreamExec │ -15)│ -------------------- ││ -------------------- │ -16)│ files: 1 ││ YieldStreamExec child: │ -17)│ format: csv ││ DataSourceExec │ -18)└───────────────────────────┘└─────────────┬─────────────┘ -19)-----------------------------┌─────────────┴─────────────┐ -20)-----------------------------│ DataSourceExec │ -21)-----------------------------│ -------------------- │ -22)-----------------------------│ files: 1 │ -23)-----------------------------│ format: parquet │ -24)-----------------------------└───────────────────────────┘ +12)└───────────────────────────┘└─────────────┬─────────────┘ +13)-----------------------------┌─────────────┴─────────────┐ +14)-----------------------------│ DataSourceExec │ +15)-----------------------------│ -------------------- │ +16)-----------------------------│ files: 1 │ +17)-----------------------------│ format: parquet │ +18)-----------------------------└───────────────────────────┘ # Query with sort merge join. @@ -1467,18 +1413,12 @@ physical_plan 09)│ c1@0 ASC ││ c1@0 ASC │ 10)└─────────────┬─────────────┘└─────────────┬─────────────┘ 11)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -12)│ YieldStreamExec ││ YieldStreamExec │ +12)│ DataSourceExec ││ DataSourceExec │ 13)│ -------------------- ││ -------------------- │ -14)│ YieldStreamExec child: ││ YieldStreamExec child: │ -15)│ DataSourceExec ││ DataSourceExec │ -16)└─────────────┬─────────────┘└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -18)│ DataSourceExec ││ DataSourceExec │ -19)│ -------------------- ││ -------------------- │ -20)│ bytes: 6040 ││ bytes: 6040 │ -21)│ format: memory ││ format: memory │ -22)│ rows: 1 ││ rows: 1 │ -23)└───────────────────────────┘└───────────────────────────┘ +14)│ bytes: 6040 ││ bytes: 6040 │ +15)│ format: memory ││ format: memory │ +16)│ rows: 1 ││ rows: 1 │ +17)└───────────────────────────┘└───────────────────────────┘ statement ok set datafusion.optimizer.prefer_hash_join = true; @@ -1587,18 +1527,12 @@ physical_plan 55)│ mode: Partial ││ mode: Partial │ 56)└─────────────┬─────────────┘└─────────────┬─────────────┘ 57)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -58)│ YieldStreamExec ││ YieldStreamExec │ +58)│ DataSourceExec ││ DataSourceExec │ 59)│ -------------------- ││ -------------------- │ -60)│ YieldStreamExec child: ││ YieldStreamExec child: │ -61)│ DataSourceExec ││ DataSourceExec │ -62)└─────────────┬─────────────┘└─────────────┬─────────────┘ -63)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -64)│ DataSourceExec ││ DataSourceExec │ -65)│ -------------------- ││ -------------------- │ -66)│ bytes: 1320 ││ bytes: 1312 │ -67)│ format: memory ││ format: memory │ -68)│ rows: 1 ││ rows: 1 │ -69)└───────────────────────────┘└───────────────────────────┘ +60)│ bytes: 1320 ││ bytes: 1312 │ +61)│ format: memory ││ format: memory │ +62)│ rows: 1 ││ rows: 1 │ +63)└───────────────────────────┘└───────────────────────────┘ # Test explain tree for UnionExec query TT @@ -2078,18 +2012,12 @@ physical_plan 51)│ predicate: a > 3 │ 52)└─────────────┬─────────────┘ 53)┌─────────────┴─────────────┐ -54)│ YieldStreamExec │ +54)│ DataSourceExec │ 55)│ -------------------- │ -56)│ YieldStreamExec child: │ -57)│ DataSourceExec │ -58)└─────────────┬─────────────┘ -59)┌─────────────┴─────────────┐ -60)│ DataSourceExec │ -61)│ -------------------- │ -62)│ bytes: 160 │ -63)│ format: memory │ -64)│ rows: 1 │ -65)└───────────────────────────┘ +56)│ bytes: 160 │ +57)│ format: memory │ +58)│ rows: 1 │ +59)└───────────────────────────┘ # clean up statement ok diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index 73dd115eed0a..e4d0b7233856 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -2088,8 +2088,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[min(t.a) FILTER (WHERE t.a > Int64(1))@0 as x] 02)--AggregateExec: mode=Single, gby=[], aggr=[min(t.a) FILTER (WHERE t.a > Int64(1))] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index f9deb3d69b16..e6879496427b 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2024,10 +2024,8 @@ physical_plan 09)----------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)] -12)----------------------YieldStreamExec child=DataSourceExec -13)------------------------DataSourceExec: partitions=1, partition_sizes=[3] -14)----------------------YieldStreamExec child=DataSourceExec -15)------------------------DataSourceExec: partitions=1, partition_sizes=[3] +12)----------------------DataSourceExec: partitions=1, partition_sizes=[3] +13)----------------------DataSourceExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is DataSourceExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2072,8 +2070,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@0 as a] 02)--SortExec: expr=[b@1 DESC], preserve_partitioning=[false] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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], file_type=csv, has_header=true +03)----DataSourceExec: 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], file_type=csv, has_header=true # Final plan shouldn't have SortExec c ASC, # because table already satisfies this ordering. @@ -2279,8 +2276,7 @@ logical_plan 02)--TableScan: multiple_ordered_table projection=[a0, a, b, c, d] physical_plan 01)SortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: 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]], file_type=csv, has_header=true +02)--DataSourceExec: 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]], file_type=csv, has_header=true query TT EXPLAIN SELECT a, b, ARRAY_AGG(d ORDER BY d) @@ -2465,8 +2461,7 @@ physical_plan 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) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T? @@ -2496,8 +2491,7 @@ physical_plan 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) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)] 03)----SortExec: expr=[amount@1 DESC], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2542,8 +2536,7 @@ physical_plan 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) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted 03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST, amount@1 DESC], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort @@ -2581,8 +2574,7 @@ physical_plan 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) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=PartiallySorted([0]) 03)----SortExec: TopK(fetch=10), expr=[country@1 ASC NULLS LAST, amount@2 DESC], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: 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, @@ -2619,8 +2611,7 @@ physical_plan 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) ORDER BY [s.country DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted 03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2656,8 +2647,7 @@ physical_plan 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) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted 03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST, amount@1 DESC], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort @@ -2754,8 +2744,7 @@ physical_plan 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) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 03)----SortExec: expr=[amount@1 DESC], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?RR rowsort SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -2786,8 +2775,7 @@ physical_plan 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) ORDER BY [sales_global.amount ASC NULLS LAST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS amounts, @@ -2819,8 +2807,7 @@ physical_plan 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) ORDER BY [sales_global.amount ASC NULLS LAST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query TRR? SELECT country, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, @@ -2850,8 +2837,7 @@ physical_plan 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) ORDER BY [sales_global.ts DESC NULLS FIRST], array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@2 ASC NULLS LAST], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query TR? SELECT country, SUM(amount ORDER BY ts DESC) AS sum1, @@ -2884,8 +2870,7 @@ logical_plan 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=[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(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2918,8 +2903,7 @@ logical_plan 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=[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(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2958,10 +2942,8 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------------YieldStreamExec child=DataSourceExec -10)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)------------DataSourceExec: 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 @@ -3006,8 +2988,7 @@ physical_plan 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) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3043,8 +3024,7 @@ physical_plan 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) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query TRR @@ -3080,8 +3060,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], 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]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3107,8 +3086,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], 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]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3133,8 +3111,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 @@ -3158,8 +3135,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------SortExec: expr=[ts@0 DESC], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 @@ -3183,8 +3159,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 05)--------SortExec: expr=[amount@0 ASC NULLS LAST], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3214,8 +3189,7 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 08)--------------SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[true] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query T? SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3251,8 +3225,7 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 08)--------------SortExec: expr=[amount@1 DESC], preserve_partitioning=[true] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -3452,8 +3425,7 @@ physical_plan 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)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query IRI SELECT s.sn, s.amount, 2*s.sn @@ -3522,11 +3494,9 @@ physical_plan 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)--------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1, projection=[amount@1, sn@2, amount@3] -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] -11)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -12)------------------YieldStreamExec child=DataSourceExec -13)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +11)------------------DataSourceExec: partitions=1, partition_sizes=[1] query IRR SELECT r.sn, SUM(l.amount), r.amount @@ -3673,8 +3643,7 @@ physical_plan 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)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] query ITIPTRR @@ -3857,8 +3826,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[last_value(foo.x)] 04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query I SELECT FIRST_VALUE(x) @@ -3880,8 +3848,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[first_value(foo.x)] 04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] # Since both ordering requirements are satisfied, there shouldn't be # any SortExec in the final plan. @@ -3902,8 +3869,7 @@ physical_plan 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) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], first_value(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: 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]], file_type=csv, has_header=true +07)------------DataSourceExec: 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]], file_type=csv, has_header=true query II rowsort SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, @@ -4015,8 +3981,7 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # drop table multiple_ordered_table_with_pk statement ok @@ -4057,8 +4022,7 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -4100,8 +4064,7 @@ physical_plan 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)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4164,12 +4127,10 @@ physical_plan 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)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true -07)----ProjectionExec: expr=[c@0 as c, sum(multiple_ordered_table_with_pk.d)@1 as sum1] -08)------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -09)--------YieldStreamExec child=DataSourceExec -10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, 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)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # we do not generate physical plan for Repartition yet (e.g Distribute By queries). query TT @@ -4283,8 +4244,7 @@ physical_plan 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)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: 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; @@ -4305,8 +4265,7 @@ physical_plan 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(x@0 AS Float64) as alias1], aggr=[] -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. statement ok @@ -4394,8 +4353,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[date_part(MONTH, ts@0) as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], file_type=csv, has_header=false +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], file_type=csv, has_header=false query I SELECT extract(month from ts) as months @@ -4511,8 +4469,7 @@ physical_plan 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)------------------------YieldStreamExec child=DataSourceExec -14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], file_type=csv, has_header=true +13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], file_type=csv, has_header=true # Use PostgreSQL dialect statement ok @@ -4685,8 +4642,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([c2@0], 8), input_partitions=8 06)----------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[max(timestamp_table.t1)], lim=[4] 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=4 -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: 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], file_type=csv, has_header=true +08)--------------DataSourceExec: 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], file_type=csv, has_header=true # Clean up statement ok @@ -5220,10 +5176,8 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }, ts@0, 946684800000000000) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))], aggr=[count(keywords_stream.keyword)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(keyword@0, keyword@1)] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)--------YieldStreamExec child=DataSourceExec -08)----------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query PI SELECT @@ -5659,8 +5613,7 @@ logical_plan 02)--TableScan: t projection=[a, b, c, 😀] physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c, 😀@3 as 😀], aggr=[] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] query TT explain select * from (select a, b from t) as c group by a, b; @@ -5671,8 +5624,7 @@ logical_plan 03)----TableScan: t projection=[a, b] physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table t; diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 108c844f20b4..ed0e675f7715 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -284,6 +284,7 @@ datafusion.format.timestamp_tz_format NULL datafusion.format.types_info false datafusion.optimizer.allow_symmetric_joins_without_pruning true datafusion.optimizer.default_filter_selectivity 20 +datafusion.optimizer.enable_add_yield_for_pipeline_break false datafusion.optimizer.enable_distinct_aggregation_soft_limit true datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.enable_topk_aggregation true @@ -394,6 +395,7 @@ datafusion.format.timestamp_tz_format NULL Timestamp format for timestamp with t datafusion.format.types_info false Show types in visual representation batches datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. datafusion.optimizer.default_filter_selectivity 20 The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). +datafusion.optimizer.enable_add_yield_for_pipeline_break false When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. Default: false (disabled). datafusion.optimizer.enable_distinct_aggregation_soft_limit true When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index de3116d5ded9..8a9c01d36308 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -73,8 +73,7 @@ physical_plan 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)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I INSERT INTO table_without_values SELECT @@ -134,8 +133,7 @@ physical_plan 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)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true @@ -186,8 +184,7 @@ physical_plan 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)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I @@ -222,8 +219,7 @@ logical_plan physical_plan 01)DataSinkExec: sink=MemoryTable (partitions=1) 02)--SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query I 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 8ac736576c49..24982dfc28a7 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -130,8 +130,7 @@ physical_plan 01)DataSinkExec: sink=CsvSink(file_groups=[]) 02)--SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] 03)----ProjectionExec: expr=[column1@0 as a, column2@1 as b] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query I INSERT INTO ordered_insert_test values (5, 1), (4, 2), (7,7), (7,8), (7,9), (7,10), (3, 3), (2, 4), (1, 5); @@ -365,8 +364,7 @@ physical_plan 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)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I INSERT INTO table_without_values SELECT @@ -427,8 +425,7 @@ physical_plan 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)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true @@ -465,8 +462,7 @@ logical_plan physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) 02)--SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query I insert into table_without_values select c1 from aggregate_test_100 order by c1; diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 6b3e3f28f903..19763ab0083f 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -698,11 +698,9 @@ logical_plan 03)--TableScan: t2 projection=[t2_id, t2_name, t2_int] physical_plan 01)CrossJoinExec -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table IF EXISTS t1; @@ -939,10 +937,8 @@ physical_plan 01)CrossJoinExec 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: name@1 = Alice OR name@1 = Bob -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)--YieldStreamExec child=DataSourceExec -07)----DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)--DataSourceExec: partitions=1, partition_sizes=[1] # expect no row for Carol query ITT @@ -1444,9 +1440,8 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=4 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] -13)------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)------DataSourceExec: partitions=1, partition_sizes=[1] query III SELECT * diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 8d368eba43be..ccecb9494331 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1344,11 +1344,9 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)--------------YieldStreamExec child=DataSourceExec -11)----------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------------DataSourceExec: partitions=1, partition_sizes=[1] # Join on struct query TT @@ -1404,11 +1402,9 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)----------------YieldStreamExec child=DataSourceExec -12)------------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1435,11 +1431,9 @@ physical_plan 08)--------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] 09)----------------CoalesceBatchesExec: target_batch_size=2 10)------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] -13)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -14)----------------------YieldStreamExec child=DataSourceExec -15)------------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)----------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -2750,14 +2744,12 @@ physical_plan 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)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)--SortExec: expr=[c1@0 ASC], preserve_partitioning=[true] -09)----CoalesceBatchesExec: target_batch_size=2 -10)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------YieldStreamExec child=DataSourceExec -13)------------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)--SortExec: expr=[c1@0 ASC], preserve_partitioning=[true] +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)----------DataSourceExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_date32 inner sort merge join on data type (Date32) query DDRTDDRT rowsort @@ -2784,14 +2776,12 @@ physical_plan 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)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)----SortExec: expr=[c3@2 ASC], preserve_partitioning=[true] -11)------CoalesceBatchesExec: target_batch_size=2 -12)--------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 -13)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -14)------------YieldStreamExec child=DataSourceExec -15)--------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----SortExec: expr=[c3@2 ASC], preserve_partitioning=[true] +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)------------DataSourceExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_decimal right join on data type (Decimal) query DDRTDDRT rowsort @@ -2845,12 +2835,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: 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 @@ -2883,12 +2871,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: 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 @@ -2942,12 +2928,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: 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 @@ -2980,12 +2964,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: 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 @@ -3040,12 +3022,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: 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 @@ -3059,12 +3039,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: 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 @@ -3116,12 +3094,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: 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 @@ -3135,12 +3111,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)----------YieldStreamExec child=DataSourceExec -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: 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 @@ -3299,16 +3273,14 @@ physical_plan 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)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: 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], file_type=csv, has_header=true -12)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -13)--------CoalesceBatchesExec: target_batch_size=2 -14)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -15)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -16)--------------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] -17)----------------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] -18)------------------YieldStreamExec child=DataSourceExec -19)--------------------DataSourceExec: 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], file_type=csv, has_header=true +10)------------------DataSourceExec: 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], file_type=csv, has_header=true +11)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] +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)------------------DataSourceExec: 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], file_type=csv, has_header=true statement ok set datafusion.optimizer.prefer_hash_join = true; @@ -3485,13 +3457,11 @@ physical_plan 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)------------------------YieldStreamExec child=DataSourceExec -14)--------------------------DataSourceExec: 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], file_type=csv, has_header=true -15)------------------CoalesceBatchesExec: target_batch_size=2 -16)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -17)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -18)------------------------YieldStreamExec child=DataSourceExec -19)--------------------------DataSourceExec: 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], file_type=csv, has_header=true +13)------------------------DataSourceExec: 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], file_type=csv, 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)------------------------DataSourceExec: 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], file_type=csv, has_header=true query TT EXPLAIN SELECT * @@ -3901,9 +3871,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] 04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------DataSourceExec: partitions=1, partition_sizes=[1] @@ -4020,9 +3989,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] 04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------DataSourceExec: partitions=1, partition_sizes=[1] # Test CROSS JOIN LATERAL syntax (planning) @@ -4127,12 +4095,10 @@ physical_plan 03)----AggregateExec: mode=Single, gby=[sn@1 as sn, ts@0 as ts, amount@2 as amount, currency@3 as currency], aggr=[last_value(e.rate)] 04)------CoalesceBatchesExec: target_batch_size=3 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: partitions=1, partition_sizes=[0] -08)----------CoalesceBatchesExec: target_batch_size=3 -09)------------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] -10)--------------YieldStreamExec child=DataSourceExec -11)----------------DataSourceExec: partitions=1, partition_sizes=[0] +06)----------DataSourceExec: partitions=1, partition_sizes=[0] +07)----------CoalesceBatchesExec: target_batch_size=3 +08)------------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] +09)--------------DataSourceExec: partitions=1, partition_sizes=[0] statement ok DROP TABLE sales_global; @@ -4158,11 +4124,9 @@ logical_plan 04)----TableScan: right_table projection=[x, y, z] physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=a@0 < x@1 -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)--SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] +04)----DataSourceExec: partitions=1, partition_sizes=[0] query TT EXPLAIN SELECT * FROM left_table JOIN right_table ON left_table.a 3, projection=[a@0] 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] -11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------YieldStreamExec child=DataSourceExec -13)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] +10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select * from test where a in (select a from test where b > 3) order by c desc nulls last; @@ -4496,12 +4454,10 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=3 06)----------FilterExec: b@1 > 3, projection=[a@0] 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] -11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------YieldStreamExec child=DataSourceExec -13)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] +10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------DataSourceExec: partitions=1, partition_sizes=[1] query III select * from test where a in (select a from test where b > 3) order by c desc nulls first; @@ -4625,10 +4581,8 @@ logical_plan 05)----TableScan: lineitem projection=[c1] physical_plan 01)CrossJoinExec -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)--YieldStreamExec child=DataSourceExec -05)----DataSourceExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT * FROM lineitem JOIN lineitem as lineitem2 USING (c1) @@ -4746,11 +4700,9 @@ logical_plan 05)------EmptyRelation physical_plan 01)CrossJoinExec -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)--ProjectionExec: expr=[1 as Int64(1)] -05)----YieldStreamExec child=PlaceholderRowExec -06)------PlaceholderRowExec +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--ProjectionExec: expr=[1 as Int64(1)] +04)----PlaceholderRowExec statement count 0 drop table j1; diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index ea243916c291..b46b8c49d662 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -63,8 +63,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]}, file_type=json +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]}, file_type=json query ? SELECT mycol FROM single_nan diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index ae178f9feaaa..2f8944f462a1 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -379,8 +379,7 @@ physical_plan 07)------------GlobalLimitExec: skip=6, fetch=3 08)--------------CoalesceBatchesExec: target_batch_size=8192, fetch=9 09)----------------FilterExec: a@0 > 3 -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query I SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); @@ -410,8 +409,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: partitions=1 +06)----------DataSourceExec: partitions=1 statement ok set datafusion.explain.show_sizes = true; @@ -643,8 +641,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(ordered_table.a)] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true # Applying offset & limit when multiple streams from union # the plan must still have a global limit to apply the offset @@ -670,13 +667,11 @@ physical_plan 04)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] 05)--------ProjectionExec: expr=[CAST(c@0 AS Int64) as c] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true -09)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] -10)--------ProjectionExec: expr=[CAST(d@0 AS Int64) as c] -11)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)------------YieldStreamExec child=DataSourceExec -13)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[d], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +08)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] +09)--------ProjectionExec: expr=[CAST(d@0 AS Int64) as c] +10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[d], file_type=csv, has_header=true # Applying LIMIT & OFFSET to subquery. query III @@ -749,10 +744,8 @@ physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] 02)--GlobalLimitExec: skip=0, fetch=10 03)----CrossJoinExec -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1], fetch=1 -06)------YieldStreamExec child=DataSourceExec -07)--------DataSourceExec: partitions=1, partition_sizes=[1], fetch=10 +04)------DataSourceExec: partitions=1, partition_sizes=[1], fetch=1 +05)------DataSourceExec: partitions=1, partition_sizes=[1], fetch=10 query IIII @@ -775,10 +768,8 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=0, fetch=2 02)--CrossJoinExec -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1], fetch=2 -05)----YieldStreamExec child=DataSourceExec -06)------DataSourceExec: partitions=1, partition_sizes=[1], fetch=2 +03)----DataSourceExec: partitions=1, partition_sizes=[1], fetch=2 +04)----DataSourceExec: partitions=1, partition_sizes=[1], fetch=2 statement ok drop table testSubQueryLimit; @@ -863,8 +854,7 @@ physical_plan 02)--SortExec: TopK(fetch=1000), expr=[part_key@1 ASC NULLS LAST], preserve_partitioning=[false] 03)----ProjectionExec: expr=[1 as foo, part_key@0 as part_key] 04)------CoalescePartitionsExec: fetch=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..794]]}, projection=[part_key], limit=1, file_type=parquet +05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..794]]}, projection=[part_key], limit=1, file_type=parquet query I with selection as ( diff --git a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt index d18b151dffea..e8700b1fea27 100644 --- a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt +++ b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt @@ -136,8 +136,7 @@ physical_plan 02)--SortExec: expr=[a_str@0 ASC NULLS LAST, b@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(a@0 AS Utf8) as a_str, b@1 as b] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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], file_type=csv, has_header=true +05)--------DataSourceExec: 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], file_type=csv, has_header=true # We cannot determine a+b is ordered from the # invariant [a ASC, b ASC] is satisfied. Hence @@ -153,8 +152,7 @@ logical_plan 02)--TableScan: multiple_ordered_table projection=[a, b] physical_plan 01)SortExec: expr=[a@0 + b@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: 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], file_type=csv, has_header=true +02)--DataSourceExec: 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], file_type=csv, 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. @@ -173,5 +171,4 @@ physical_plan 02)--SortExec: expr=[sum_expr@0 ASC NULLS LAST], preserve_partitioning=[true] 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)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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], file_type=csv, has_header=true +05)--------DataSourceExec: 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], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 6aed9bfd38d7..3fc90a6459f2 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -166,8 +166,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] 02)--SortExec: expr=[c2@1 ASC NULLS LAST, c3@2 ASC NULLS LAST], preserve_partitioning=[false] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true query II SELECT c2, c3 FROM aggregate_test_100 ORDER BY c2, c3, c2 @@ -706,8 +705,7 @@ logical_plan 02)--TableScan: t1 projection=[id, name] physical_plan 01)SortExec: expr=[id@0 DESC, name@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] query IT SELECT * FROM t1 ORDER BY id ASC, id, name, id DESC; @@ -726,8 +724,7 @@ logical_plan 02)--TableScan: t1 projection=[id, name] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST, name@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # Minimal reproduction of issue 5970 @@ -802,17 +799,15 @@ physical_plan 08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 09)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] 10)------------------ProjectionExec: expr=[column1@0 as t] -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] -13)------ProjectionExec: expr=[1 as m, t@0 as t] -14)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] -15)----------CoalesceBatchesExec: target_batch_size=8192 -16)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 -17)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -18)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] -19)------------------ProjectionExec: expr=[column1@0 as t] -20)--------------------YieldStreamExec child=DataSourceExec -21)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)------ProjectionExec: expr=[1 as m, t@0 as t] +13)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] +14)----------CoalesceBatchesExec: target_batch_size=8192 +15)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 +16)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +17)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] +18)------------------ProjectionExec: expr=[column1@0 as t] +19)--------------------DataSourceExec: partitions=1, partition_sizes=[1] ##### # Multi column sorting with lists @@ -1052,8 +1047,7 @@ physical_plan 02)--SortExec: TopK(fetch=5), expr=[c_str@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(c@0 AS Utf8View) as c_str] 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true # Casting from numeric to numeric types preserves the ordering @@ -1119,8 +1113,7 @@ physical_plan 02)--SortExec: TopK(fetch=5), expr=[abs_c@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[abs(c@0) as abs_c] 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table ordered_table; @@ -1204,8 +1197,7 @@ physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] 02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true statement ok set datafusion.execution.use_row_number_estimates_to_optimize_partitioning = true; @@ -1224,8 +1216,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] 02)--SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true statement ok set datafusion.execution.use_row_number_estimates_to_optimize_partitioning = false; @@ -1245,8 +1236,7 @@ physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] 02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true # Test: inputs into union with different orderings @@ -1271,12 +1261,10 @@ physical_plan 02)--UnionExec 03)----SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a@2 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] 04)------ProjectionExec: expr=[b@1 as b, c@2 as c, a@0 as a, NULL as a0, d@3 as d] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[c@2 ASC NULLS LAST], file_type=csv, has_header=true -07)----SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a0@3 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] -08)------ProjectionExec: expr=[b@1 as b, c@2 as c, NULL as a, a0@0 as a0, d@3 as d] -09)--------YieldStreamExec child=DataSourceExec -10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, b, c, d], output_ordering=[c@2 ASC NULLS LAST], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[c@2 ASC NULLS LAST], file_type=csv, has_header=true +06)----SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a0@3 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] +07)------ProjectionExec: expr=[b@1 as b, c@2 as c, NULL as a, a0@0 as a0, d@3 as d] +08)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, b, c, d], output_ordering=[c@2 ASC NULLS LAST], file_type=csv, has_header=true # Test: run the query from above query IIIII diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 84db2d1280a7..abc6fdab3c8a 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -91,8 +91,7 @@ logical_plan physical_plan 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], preserve_partitioning=[true] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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], file_type=parquet +03)----DataSourceExec: 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], file_type=parquet # Tear down test_table: statement ok @@ -144,8 +143,7 @@ logical_plan physical_plan 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], preserve_partitioning=[true] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: 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], file_type=parquet +03)----DataSourceExec: 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], file_type=parquet # Perform queries using MIN and MAX diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 9bb2c5a2afbb..1b6ae13fbe77 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -86,8 +86,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] # When filter pushdown *is* enabled, ParquetExec can filter exactly, @@ -115,8 +114,7 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: b@1 > 2, projection=[a@0] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +06)----------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] # also test querying on columns that are not in all the files query T @@ -136,8 +134,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2 AND a_null_count@3 != row_count@2, required_guarantees=[] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2 AND a_null_count@3 != row_count@2, required_guarantees=[] query I @@ -156,8 +153,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] 02)--SortExec: expr=[b@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] ## cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt index 9a48c1ac003f..a10243f62720 100644 --- a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt @@ -266,5 +266,4 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [constant_col@0 ASC NULLS LAST] 02)--SortExec: expr=[constant_col@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[constant_col], file_type=parquet +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[constant_col], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 3b7dbe934cfa..b4b31fa78a69 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -763,22 +763,19 @@ physical_plan 02)--ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey, ps_suppkey@0 as ps_suppkey] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, p_partkey@2)], projection=[ps_suppkey@1, l_extendedprice@2, l_discount@3, p_partkey@4] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)--------CoalesceBatchesExec: target_batch_size=8192 -08)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] -09)------------CoalesceBatchesExec: target_batch_size=8192 -10)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -11)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)------------------YieldStreamExec child=DataSourceExec -13)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true -14)------------CoalesceBatchesExec: target_batch_size=8192 -15)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -16)----------------CoalesceBatchesExec: target_batch_size=8192 -17)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -18)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)----------------------YieldStreamExec child=DataSourceExec -20)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--------CoalesceBatchesExec: target_batch_size=8192 +07)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] +08)------------CoalesceBatchesExec: target_batch_size=8192 +09)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +10)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true +12)------------CoalesceBatchesExec: target_batch_size=8192 +13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +14)----------------CoalesceBatchesExec: target_batch_size=8192 +15)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/references.slt b/datafusion/sqllogictest/test_files/references.slt index bbb3c8b52cdf..0e72c5e5a29e 100644 --- a/datafusion/sqllogictest/test_files/references.slt +++ b/datafusion/sqllogictest/test_files/references.slt @@ -107,8 +107,7 @@ logical_plan physical_plan 01)SortExec: expr=[....@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[....@0 as ...., ....@0 as c3] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: 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 05d305122b5a..70666346e2ca 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -48,8 +48,7 @@ physical_plan 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)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # disable round robin repartitioning statement ok @@ -66,8 +65,7 @@ physical_plan 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)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # Cleanup diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index aa0bcf377627..2b30de572c8c 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -102,8 +102,7 @@ physical_plan 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: column1@0 != 42 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:272..538, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..278], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:278..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:272..538, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..278], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:278..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 4e373a7ad6d0..aa14faf984e4 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -565,10 +565,8 @@ logical_plan 06)------TableScan: foo projection=[column2] physical_plan 01)CrossJoinExec -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)--YieldStreamExec child=DataSourceExec -05)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--DataSourceExec: partitions=1, partition_sizes=[1] # select NaNs query BBBB @@ -1381,8 +1379,7 @@ logical_plan 02)--TableScan: table1 projection=[a] physical_plan 01)SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # ambiguous column references in on join query error DataFusion error: Schema error: Ambiguous reference to unqualified field a @@ -1531,8 +1528,7 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: a@1 = 0 OR b@2 = 0 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: 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], file_type=csv, has_header=true +06)----------DataSourceExec: 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], file_type=csv, has_header=true # When ordering lost during projection, we shouldn't keep the SortExec. # in the final physical plan. @@ -1560,8 +1556,7 @@ physical_plan 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)------------ProjectionExec: expr=[c2@0 as c2] 08)--------------SortExec: TopK(fetch=4), expr=[c1@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c1], file_type=csv, has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c1], file_type=csv, has_header=true # FilterExec can track equality of non-column expressions. # plan below shouldn't have a SortExec because given column 'a' is ordered. diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index 950791bfe306..c17fe8dfc7e6 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -39,11 +39,9 @@ logical_plan physical_plan 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], preserve_partitioning=[false] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)--SortExec: expr=[a@0 ASC], preserve_partitioning=[false] -06)----YieldStreamExec child=DataSourceExec -07)------DataSourceExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)--SortExec: expr=[a@0 ASC], preserve_partitioning=[false] +05)----DataSourceExec: partitions=1, partition_sizes=[1] # inner join with join filter query TITI rowsort diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 65152b621e42..796570633f67 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -210,10 +210,9 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] -13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)--------DataSourceExec: partitions=1, partition_sizes=[1] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -246,10 +245,9 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] -13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)--------DataSourceExec: partitions=1, partition_sizes=[1] 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 @@ -282,10 +280,9 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] -13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)--------DataSourceExec: partitions=1, partition_sizes=[1] 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 @@ -321,10 +318,9 @@ physical_plan 10)------------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------------------YieldStreamExec child=DataSourceExec -14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] -15)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)--------DataSourceExec: partitions=1, partition_sizes=[1] +13)------------------------DataSourceExec: partitions=1, partition_sizes=[1] +14)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)--------DataSourceExec: partitions=1, partition_sizes=[1] 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 diff --git a/datafusion/sqllogictest/test_files/subquery_sort.slt b/datafusion/sqllogictest/test_files/subquery_sort.slt index 4f873d259df2..d993515f4de9 100644 --- a/datafusion/sqllogictest/test_files/subquery_sort.slt +++ b/datafusion/sqllogictest/test_files/subquery_sort.slt @@ -67,8 +67,7 @@ physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] 02)--SortExec: expr=[c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST, c9@3 ASC NULLS LAST], preserve_partitioning=[false] 03)----SortExec: TopK(fetch=2), expr=[c1@0 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true query TI @@ -103,8 +102,7 @@ physical_plan 03)----ProjectionExec: expr=[c1@0 as c1, rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as r, c3@1 as c3, c9@2 as c9] 04)------BoundedWindowAggExec: wdw=[rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "rank() ORDER BY [sink_table.c1 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(Utf8View(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 DESC], preserve_partitioning=[false] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3, c9], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3, c9], file_type=csv, has_header=true #Test with utf8view for window function statement ok @@ -130,8 +128,7 @@ physical_plan 03)----ProjectionExec: expr=[c1@0 as c1, rank() ORDER BY [sink_table_with_utf8view.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as r, c3@1 as c3, c9@2 as c9] 04)------BoundedWindowAggExec: wdw=[rank() ORDER BY [sink_table_with_utf8view.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "rank() ORDER BY [sink_table_with_utf8view.c1 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(Utf8View(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 DESC], preserve_partitioning=[false] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok DROP TABLE sink_table_with_utf8view; @@ -157,8 +154,7 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true query TI diff --git a/datafusion/sqllogictest/test_files/topk.slt b/datafusion/sqllogictest/test_files/topk.slt index 839293dc0b95..ce23fe26528c 100644 --- a/datafusion/sqllogictest/test_files/topk.slt +++ b/datafusion/sqllogictest/test_files/topk.slt @@ -85,8 +85,7 @@ logical_plan 02)--TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] physical_plan 01)SortExec: TopK(fetch=5), expr=[c13@12 DESC], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: 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], file_type=csv, has_header=true +02)--DataSourceExec: 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], file_type=csv, has_header=true @@ -317,8 +316,7 @@ explain select number, letter, age from partial_sorted order by number desc, let ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 ASC NULLS LAST, age@2 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC, letter@1 ASC NULLS LAST] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet # Explain variations of the above query with different orderings, and different sort prefixes. @@ -328,32 +326,28 @@ explain select number, letter, age from partial_sorted order by age desc limit 3 ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[age@2 DESC], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet query TT explain select number, letter, age from partial_sorted order by number desc, letter desc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet query TT explain select number, letter, age from partial_sorted order by number asc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet query TT explain select number, letter, age from partial_sorted order by letter asc, number desc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[letter@1 ASC NULLS LAST, number@0 DESC], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet # Explicit NULLS ordering cases (reversing the order of the NULLS on the number and letter orderings) query TT @@ -361,16 +355,14 @@ explain select number, letter, age from partial_sorted order by number desc, let ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 ASC], preserve_partitioning=[false], sort_prefix=[number@0 DESC] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet query TT explain select number, letter, age from partial_sorted order by number desc NULLS LAST, letter asc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC NULLS LAST, letter@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--YieldStreamExec child=DataSourceExec -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet # Verify that the sort prefix is correctly computed on the normalized ordering (removing redundant aliased columns) @@ -380,8 +372,7 @@ explain select number, letter, age, number as column4, letter as column5 from pa physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, column4@3 DESC, letter@1 ASC NULLS LAST, column5@4 ASC NULLS LAST, age@2 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC, letter@1 ASC NULLS LAST] 02)--ProjectionExec: expr=[number@0 as number, letter@1 as letter, age@2 as age, number@0 as column4, letter@1 as column5] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet # Verify that the sort prefix is correctly computed over normalized, order-maintaining projections (number + 1, number, number + 1, age) query TT @@ -393,8 +384,7 @@ physical_plan 03)----ProjectionExec: expr=[__common_expr_1@0 as number_plus, number@1 as number, __common_expr_1@0 as other_number_plus, age@2 as age] 04)------ProjectionExec: expr=[CAST(number@0 AS Int64) + 1 as __common_expr_1, number@0 as number, age@1 as age] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, age], output_ordering=[number@0 DESC], file_type=parquet +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, age], output_ordering=[number@0 DESC], file_type=parquet # Cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 9cdfef802b87..d549f555f9d8 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -239,13 +239,10 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 05)--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 06)----------UnionExec -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------------YieldStreamExec child=DataSourceExec +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------------ProjectionExec: expr=[name@0 || _new as name] 10)--------------DataSourceExec: partitions=1, partition_sizes=[1] -11)------------ProjectionExec: expr=[name@0 || _new as name] -12)--------------YieldStreamExec child=DataSourceExec -13)----------------DataSourceExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort @@ -318,25 +315,23 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -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)----------DataSourceExec: 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=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] -18)--------CoalescePartitionsExec -19)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -20)------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -21)--------------CoalesceBatchesExec: target_batch_size=2 -22)----------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -23)------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -24)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -25)----------------------YieldStreamExec child=DataSourceExec -26)------------------------DataSourceExec: partitions=1, partition_sizes=[1] -27)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)----------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +12)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)----------DataSourceExec: partitions=1, partition_sizes=[1] +14)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] +15)----CoalesceBatchesExec: target_batch_size=2 +16)------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] +17)--------CoalescePartitionsExec +18)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +19)------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +20)--------------CoalesceBatchesExec: target_batch_size=2 +21)----------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +22)------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +23)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +24)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +25)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +26)----------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort @@ -390,22 +385,20 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------DataSourceExec: partitions=1, partition_sizes=[1] -14)--CoalesceBatchesExec: target_batch_size=2 -15)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)] -16)------CoalescePartitionsExec -17)--------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] -18)----------CoalesceBatchesExec: target_batch_size=2 -19)------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -20)--------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -21)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -22)------------------YieldStreamExec child=DataSourceExec -23)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -24)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -25)--------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)--------DataSourceExec: partitions=1, partition_sizes=[1] +13)--CoalesceBatchesExec: target_batch_size=2 +14)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)] +15)------CoalescePartitionsExec +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)------------------DataSourceExec: partitions=1, partition_sizes=[1] +22)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +23)--------DataSourceExec: partitions=1, partition_sizes=[1] # union_upcast_types query TT @@ -424,13 +417,11 @@ physical_plan 03)----SortExec: TopK(fetch=5), expr=[c9@1 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[c1@0 as c1, CAST(c9@1 AS Decimal128(20, 0)) as c9] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true -08)----SortExec: TopK(fetch=5), expr=[c9@1 DESC], preserve_partitioning=[true] -09)------ProjectionExec: expr=[c1@0 as c1, CAST(c3@1 AS Decimal128(20, 0)) as c9] -10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)----------YieldStreamExec child=DataSourceExec -12)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true +07)----SortExec: TopK(fetch=5), expr=[c9@1 DESC], preserve_partitioning=[true] +08)------ProjectionExec: expr=[c1@0 as c1, CAST(c3@1 AS Decimal128(20, 0)) as c9] +09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true query TR SELECT c1, c9 FROM aggregate_test_100 UNION ALL SELECT c1, c3 FROM aggregate_test_100 ORDER BY c9 DESC LIMIT 5 @@ -467,15 +458,13 @@ physical_plan 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)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] -11)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] -12)--------CoalesceBatchesExec: target_batch_size=2 -13)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -14)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -15)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)----------------YieldStreamExec child=DataSourceExec -17)------------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: 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)----------------DataSourceExec: partitions=1, partition_sizes=[1] # Union with limit push down 3 children test case query TT @@ -529,16 +518,13 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=2 13)------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------------------YieldStreamExec child=DataSourceExec -16)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true -17)----ProjectionExec: expr=[1 as cnt] -18)------YieldStreamExec child=PlaceholderRowExec -19)--------PlaceholderRowExec -20)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] -21)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) 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 }], mode=[Sorted] -22)--------ProjectionExec: expr=[1 as c1] -23)----------YieldStreamExec child=PlaceholderRowExec -24)------------PlaceholderRowExec +15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true +16)----ProjectionExec: expr=[1 as cnt] +17)------PlaceholderRowExec +18)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] +19)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) 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 }], mode=[Sorted] +20)--------ProjectionExec: expr=[1 as c1] +21)----------PlaceholderRowExec ######## @@ -673,12 +659,10 @@ physical_plan 02)--ProjectionExec: expr=[count(Int64(1))@1 as count, n@0 as n] 03)----AggregateExec: mode=SinglePartitioned, gby=[n@0 as n], aggr=[count(Int64(1))], ordering_mode=Sorted 04)------ProjectionExec: expr=[5 as n] -05)--------YieldStreamExec child=PlaceholderRowExec -06)----------PlaceholderRowExec -07)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] -08)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] -09)------YieldStreamExec child=PlaceholderRowExec -10)--------PlaceholderRowExec +05)--------PlaceholderRowExec +06)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] +07)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] +08)------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/11409 @@ -932,11 +916,9 @@ physical_plan 03)----SortExec: expr=[y@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[CAST(y@0 AS Int64) as y] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)----SortExec: expr=[y@0 ASC NULLS LAST], preserve_partitioning=[false] -09)------YieldStreamExec child=DataSourceExec -10)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----SortExec: expr=[y@0 ASC NULLS LAST], preserve_partitioning=[false] +08)------DataSourceExec: partitions=1, partition_sizes=[1] # optimize_subquery_sort in create_relation removes Sort so the result is not sorted. query I diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 9bd7188e878d..c86921012f9b 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -280,17 +280,13 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[max(d.a)], ordering_mode=Sorted 08)--------------UnionExec 09)----------------ProjectionExec: expr=[1 as a, aa as b] -10)------------------YieldStreamExec child=PlaceholderRowExec -11)--------------------PlaceholderRowExec -12)----------------ProjectionExec: expr=[3 as a, aa as b] -13)------------------YieldStreamExec child=PlaceholderRowExec -14)--------------------PlaceholderRowExec -15)----------------ProjectionExec: expr=[5 as a, bb as b] -16)------------------YieldStreamExec child=PlaceholderRowExec -17)--------------------PlaceholderRowExec -18)----------------ProjectionExec: expr=[7 as a, bb as b] -19)------------------YieldStreamExec child=PlaceholderRowExec -20)--------------------PlaceholderRowExec +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 @@ -370,17 +366,13 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4 09)----------------UnionExec 10)------------------ProjectionExec: expr=[1 as a, aa as b] -11)--------------------YieldStreamExec child=PlaceholderRowExec -12)----------------------PlaceholderRowExec -13)------------------ProjectionExec: expr=[3 as a, aa as b] -14)--------------------YieldStreamExec child=PlaceholderRowExec -15)----------------------PlaceholderRowExec -16)------------------ProjectionExec: expr=[5 as a, bb as b] -17)--------------------YieldStreamExec child=PlaceholderRowExec -18)----------------------PlaceholderRowExec -19)------------------ProjectionExec: expr=[7 as a, bb as b] -20)--------------------YieldStreamExec child=PlaceholderRowExec -21)----------------------PlaceholderRowExec +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 @@ -1253,8 +1245,7 @@ physical_plan 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], preserve_partitioning=[false] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], file_type=csv, has_header=true # over_order_by_sort_keys_sorting_prefix_compacting @@ -1275,8 +1266,7 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c2@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[false] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], file_type=csv, 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 @@ -1301,8 +1291,7 @@ physical_plan 05)--------SortExec: expr=[c9@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] 06)----------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 07)------------SortExec: expr=[c2@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[false] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], file_type=csv, has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], file_type=csv, has_header=true # test_window_partition_by_order_by statement ok @@ -1332,8 +1321,7 @@ physical_plan 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)----------------------YieldStreamExec child=DataSourceExec -13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], file_type=csv, has_header=true +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], file_type=csv, has_header=true # test_window_agg_sort_reversed_plan @@ -1358,8 +1346,7 @@ physical_plan 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], preserve_partitioning=[false] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query III SELECT @@ -1400,8 +1387,7 @@ physical_plan 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], preserve_partitioning=[false] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query IIIIIII SELECT @@ -1445,8 +1431,7 @@ physical_plan 04)------SortExec: expr=[c9@0 ASC NULLS LAST], preserve_partitioning=[false] 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], preserve_partitioning=[false] -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query III @@ -1489,8 +1474,7 @@ physical_plan 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], preserve_partitioning=[false] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], file_type=csv, has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], file_type=csv, has_header=true query IIII SELECT @@ -1581,8 +1565,7 @@ physical_plan 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], preserve_partitioning=[false] -17)--------------------------------YieldStreamExec child=DataSourceExec -18)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true +17)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true query IIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -1657,8 +1640,7 @@ physical_plan 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], preserve_partitioning=[false] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true query III @@ -1702,8 +1684,7 @@ physical_plan 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], preserve_partitioning=[false] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true query III SELECT @@ -1753,8 +1734,7 @@ physical_plan 07)------------SortExec: expr=[__common_expr_1@0 DESC, c9@3 DESC, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------ProjectionExec: expr=[c3@1 + c4@2 as __common_expr_1, c2@0 as c2, c3@1 as c3, c9@3 as c9] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], file_type=csv, has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], file_type=csv, has_header=true query III @@ -1805,8 +1785,7 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=4096 11)--------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)------------------------YieldStreamExec child=DataSourceExec -14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true +13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true query I @@ -1851,8 +1830,7 @@ physical_plan 08)--------------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] 09)----------------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] 10)------------------SortExec: expr=[c3@1 DESC, c9@2 DESC, c2@0 ASC NULLS LAST], preserve_partitioning=[false] -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], file_type=csv, has_header=true +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], file_type=csv, has_header=true @@ -1891,8 +1869,7 @@ physical_plan 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)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query TI SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC @@ -2021,8 +1998,7 @@ physical_plan 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)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true statement ok set datafusion.optimizer.repartition_sorts = true; @@ -2051,8 +2027,7 @@ physical_plan 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)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true # test_window_agg_with_global_limit statement ok @@ -2073,8 +2048,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(aggregate_test_100.c13)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST], preserve_partitioning=[false] -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], file_type=csv, has_header=true query ? @@ -2139,8 +2113,7 @@ physical_plan 06)----------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] 07)------------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 }] 08)--------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, c9@3 ASC NULLS LAST, c8@2 ASC NULLS LAST], preserve_partitioning=[false] -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], file_type=csv, has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], file_type=csv, has_header=true @@ -2197,8 +2170,7 @@ physical_plan 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], preserve_partitioning=[false] 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)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], file_type=csv, has_header=true +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], file_type=csv, has_header=true query IIIII SELECT c9, @@ -2240,8 +2212,7 @@ physical_plan 05)--------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] 06)----------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] 07)------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[false] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], file_type=csv, has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], file_type=csv, has_header=true query RR SELECT SUM(c12) OVER(ORDER BY c1, c2 GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as sum1, @@ -2275,8 +2246,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 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], preserve_partitioning=[false] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2313,8 +2283,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 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], preserve_partitioning=[false] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2351,8 +2320,7 @@ physical_plan 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], preserve_partitioning=[false] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2392,8 +2360,7 @@ physical_plan 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], preserve_partitioning=[false] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2468,8 +2435,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 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], preserve_partitioning=[false] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, 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. @@ -2491,8 +2457,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 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(Decimal128(None,21,0)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[CAST(c9@1 AS Decimal128(20, 0)) + CAST(c5@0 AS Decimal128(20, 0)) DESC], preserve_partitioning=[false] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], file_type=csv, has_header=true # Ordering equivalence should be preserved during cast expression query TT @@ -2513,8 +2478,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 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], preserve_partitioning=[false] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, 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 @@ -2622,8 +2586,7 @@ physical_plan 06)----------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, count(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: false, 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(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 07)------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, count(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, 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(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(8)), is_causal: false }], mode=[Sorted] 08)--------------ProjectionExec: expr=[CAST(desc_col@2 AS Int64) as __common_expr_1, CAST(inc_col@1 AS Int64) as __common_expr_2, ts@0 as ts, inc_col@1 as inc_col, desc_col@2 as desc_col] -09)----------------YieldStreamExec child=DataSourceExec -10)------------------DataSourceExec: 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], file_type=csv, has_header=true +09)----------------DataSourceExec: 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], file_type=csv, has_header=true query IIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -2706,8 +2669,7 @@ physical_plan 02)--ProjectionExec: expr=[ts@0 as ts, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] 03)----BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }], mode=[Sorted] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true query IIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -2780,8 +2742,7 @@ physical_plan 04)------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }], mode=[Sorted] 05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }], mode=[Sorted] 06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, CAST(inc_col@1 AS Float64) as __common_expr_2, ts@0 as ts, inc_col@1 as inc_col] -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true query IIIIIIIIRR SELECT @@ -2832,8 +2793,7 @@ physical_plan 03)----ProjectionExec: expr=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col] 04)------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 05)--------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true query IIIII SELECT @@ -3100,8 +3060,7 @@ physical_plan 12)----------------------SortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 13)------------------------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] 14)--------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -15)----------------------------YieldStreamExec child=DataSourceExec -16)------------------------------DataSourceExec: 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], file_type=csv, has_header=true +15)----------------------------DataSourceExec: 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], file_type=csv, has_header=true query IIIIIIIIIIIIIII SELECT a, b, c, @@ -3187,8 +3146,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----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] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, 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 @@ -3376,8 +3334,7 @@ physical_plan 05)--------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] 06)----------WindowAggExec: wdw=[min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] 07)------------SortExec: expr=[c11@1 ASC NULLS LAST], preserve_partitioning=[false] -08)--------------YieldStreamExec child=DataSourceExec -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], file_type=csv, has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], file_type=csv, has_header=true # window1 spec is used multiple times under different aggregations. # The query should still work. @@ -3418,8 +3375,7 @@ physical_plan 03)----ProjectionExec: expr=[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min1, min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max1, c3@0 as c3] 04)------BoundedWindowAggExec: wdw=[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }, min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c12@1 ASC NULLS LAST], preserve_partitioning=[false] -06)----------YieldStreamExec child=DataSourceExec -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], file_type=csv, has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], file_type=csv, has_header=true # window2 spec is not defined statement error DataFusion error: Error during planning: The window window2 is not defined! @@ -3491,8 +3447,7 @@ physical_plan 02)--SortExec: expr=[d@4 ASC NULLS LAST], preserve_partitioning=[false] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------FilterExec: b@2 = 0 -05)--------YieldStreamExec child=DataSourceExec -06)----------DataSourceExec: 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]], file_type=csv, has_header=true +05)--------DataSourceExec: 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]], file_type=csv, has_header=true # Create an unbounded source where there is multiple orderings. @@ -3615,8 +3570,7 @@ physical_plan 01)ProjectionExec: expr=[c@0 as c, nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as nv1] 02)--GlobalLimitExec: skip=0, fetch=5 03)----WindowAggExec: wdw=[nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int32(NULL)), is_causal: false }] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true query II SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 @@ -4003,8 +3957,7 @@ physical_plan 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], preserve_partitioning=[false] -04)------YieldStreamExec child=DataSourceExec -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] # test ROW_NUMBER window function returns correct data_type query T @@ -4125,8 +4078,7 @@ physical_plan 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], preserve_partitioning=[false] -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], file_type=csv, has_header=true query III SELECT c3, @@ -5255,8 +5207,7 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------FilterExec: c1@0 = 2 OR c1@0 = 3 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5298,8 +5249,7 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5343,8 +5293,7 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=1 10)------------------FilterExec: c1@0 = 1 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------------------YieldStreamExec child=DataSourceExec -13)------------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5383,8 +5332,7 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------YieldStreamExec child=DataSourceExec -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5433,8 +5381,7 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=1 13)------------------------FilterExec: c1@0 > 1 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)----------------------------YieldStreamExec child=DataSourceExec -16)------------------------------DataSourceExec: partitions=1, partition_sizes=[1] +15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2 @@ -5485,8 +5432,7 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=1 13)------------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)----------------------------YieldStreamExec child=DataSourceExec -16)------------------------------DataSourceExec: partitions=1, partition_sizes=[1] +15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2 @@ -5546,8 +5492,7 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=1 05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT SUM(c9) OVER() as sum_c9 FROM aggregate_test_100_ordered ORDER BY sum_c9; @@ -5560,8 +5505,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as sum_c9] 02)--WindowAggExec: wdw=[sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100_ordered.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)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query TT EXPLAIN SELECT c1, MIN(c5) OVER(PARTITION BY c1) as min_c5 FROM aggregate_test_100_ordered ORDER BY c1, min_c5 DESC NULLS LAST; @@ -5578,8 +5522,7 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=1 05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT MAX(c5) OVER() as max_c5 FROM aggregate_test_100_ordered ORDER BY max_c5; @@ -5592,8 +5535,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as max_c5] 02)--WindowAggExec: wdw=[max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED 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(NULL)), is_causal: false }] -03)----YieldStreamExec child=DataSourceExec -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], file_type=csv, has_header=true query II rowsort SELECT From 2de4afb6c41c4eade6ac9db7a33b11f2d4f5676d Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 4 Jun 2025 22:06:47 +0800 Subject: [PATCH 33/74] recover testing --- datafusion/core/tests/dataframe/mod.rs | 91 ++++++-------- datafusion/core/tests/memory_limit/mod.rs | 19 ++- datafusion/core/tests/sql/explain_analyze.rs | 3 +- .../src/aggregates/no_grouping.rs | 4 +- .../physical-plan/src/aggregates/row_hash.rs | 1 - .../test_files/tpch/plans/q1.slt.part | 3 +- .../test_files/tpch/plans/q10.slt.part | 34 +++--- .../test_files/tpch/plans/q11.slt.part | 72 +++++------ .../test_files/tpch/plans/q12.slt.part | 10 +- .../test_files/tpch/plans/q13.slt.part | 14 +-- .../test_files/tpch/plans/q14.slt.part | 12 +- .../test_files/tpch/plans/q15.slt.part | 33 +++-- .../test_files/tpch/plans/q16.slt.part | 29 ++--- .../test_files/tpch/plans/q17.slt.part | 29 ++--- .../test_files/tpch/plans/q18.slt.part | 32 +++-- .../test_files/tpch/plans/q19.slt.part | 16 ++- .../test_files/tpch/plans/q2.slt.part | 113 ++++++++---------- .../test_files/tpch/plans/q20.slt.part | 69 +++++------ .../test_files/tpch/plans/q21.slt.part | 56 ++++----- .../test_files/tpch/plans/q22.slt.part | 25 ++-- .../test_files/tpch/plans/q3.slt.part | 25 ++-- .../test_files/tpch/plans/q4.slt.part | 14 +-- .../test_files/tpch/plans/q5.slt.part | 52 ++++---- .../test_files/tpch/plans/q6.slt.part | 3 +- .../test_files/tpch/plans/q7.slt.part | 56 ++++----- .../test_files/tpch/plans/q8.slt.part | 70 +++++------ .../test_files/tpch/plans/q9.slt.part | 42 +++---- 27 files changed, 412 insertions(+), 515 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 37fd2edafe50..aa36de1e555f 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -572,8 +572,7 @@ async fn test_aggregate_with_pk() -> Result<()> { physical_plan_to_string(&df).await, @r###" AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[] - YieldStreamExec child=DataSourceExec - DataSourceExec: partitions=1, partition_sizes=[1] + DataSourceExec: partitions=1, partition_sizes=[1] "### ); @@ -616,8 +615,7 @@ async fn test_aggregate_with_pk2() -> Result<()> { CoalesceBatchesExec: target_batch_size=8192 FilterExec: id@0 = 1 AND name@1 = a AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[] - YieldStreamExec child=DataSourceExec - DataSourceExec: partitions=1, partition_sizes=[1] + DataSourceExec: partitions=1, partition_sizes=[1] "### ); @@ -666,8 +664,7 @@ async fn test_aggregate_with_pk3() -> Result<()> { CoalesceBatchesExec: target_batch_size=8192 FilterExec: id@0 = 1 AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[] - YieldStreamExec child=DataSourceExec - DataSourceExec: partitions=1, partition_sizes=[1] + DataSourceExec: partitions=1, partition_sizes=[1] "### ); @@ -718,8 +715,7 @@ async fn test_aggregate_with_pk4() -> Result<()> { CoalesceBatchesExec: target_batch_size=8192 FilterExec: id@0 = 1 AggregateExec: mode=Single, gby=[id@0 as id], aggr=[] - YieldStreamExec child=DataSourceExec - DataSourceExec: partitions=1, partition_sizes=[1] + DataSourceExec: partitions=1, partition_sizes=[1] "### ); @@ -911,7 +907,7 @@ async fn window_using_aggregates() -> Result<()> { vec![col("c3")], ); - Expr::WindowFunction(w) + Expr::from(w) .null_treatment(NullTreatment::IgnoreNulls) .order_by(vec![col("c2").sort(true, true), col("c3").sort(true, true)]) .window_frame(WindowFrame::new_bounds( @@ -2592,8 +2588,7 @@ async fn test_count_wildcard_on_sort() -> Result<()> { | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] | - | | YieldStreamExec child=DataSourceExec | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+------------------------------------------------------------------------------------------------------------+ "### @@ -2615,13 +2610,11 @@ async fn test_count_wildcard_on_sort() -> Result<()> { | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] | - | | YieldStreamExec child=DataSourceExec | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+--------------------------------------------------------------------------------+ "### ); - Ok(()) } @@ -2791,21 +2784,20 @@ async fn test_count_wildcard_on_window() -> Result<()> { .await?; assert_snapshot!( - pretty_format_batches(&sql_results).unwrap(), - @r###" -+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| plan_type | plan | -+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING | -| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] | -| | TableScan: t1 projection=[a] | -| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] | -| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] | -| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] | -| | YieldStreamExec child=DataSourceExec | -| | DataSourceExec: partitions=1, partition_sizes=[1] | -| | | -+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + pretty_format_batches(&sql_results).unwrap(), + @r###" + +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING | + | | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] | + | | TableScan: t1 projection=[a] | + | physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] | + | | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] | + | | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ "### ); @@ -2826,21 +2818,20 @@ async fn test_count_wildcard_on_window() -> Result<()> { .await?; assert_snapshot!( - pretty_format_batches(&df_results).unwrap(), - @r###" -+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| plan_type | plan | -+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING | -| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] | -| | TableScan: t1 projection=[a] | -| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] | -| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] | -| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] | -| | YieldStreamExec child=DataSourceExec | -| | DataSourceExec: partitions=1, partition_sizes=[1] | -| | | -+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + pretty_format_batches(&df_results).unwrap(), + @r###" + +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING | + | | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] | + | | TableScan: t1 projection=[a] | + | physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] | + | | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] | + | | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ "### ); @@ -2933,16 +2924,14 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | | | CoalesceBatchesExec: target_batch_size=8192 | | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | YieldStreamExec child=DataSourceExec | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | | | CoalesceBatchesExec: target_batch_size=8192 | | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | YieldStreamExec child=DataSourceExec | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+---------------------------------------------------------------------------------------------------------------------------+ " @@ -2992,16 +2981,14 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | | | CoalesceBatchesExec: target_batch_size=8192 | | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | YieldStreamExec child=DataSourceExec | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | | | CoalesceBatchesExec: target_batch_size=8192 | | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | - | | YieldStreamExec child=DataSourceExec | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+---------------------------------------------------------------------------------------------------------------------------+ " diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index fbee8684f3f2..7695cc0969d8 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -294,16 +294,15 @@ async fn sort_spill_reservation() { // also merge, so we can ensure the sort could finish // given enough merging memory &[ - "+---------------+---------------------------------------------------------------------------------------------------------------+", - "| plan_type | plan |", - "+---------------+---------------------------------------------------------------------------------------------------------------+", - "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", - "| | TableScan: t projection=[a, b] |", - "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] |", - "| | YieldStreamExec child=DataSourceExec |", - "| | DataSourceExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", - "| | |", - "+---------------+---------------------------------------------------------------------------------------------------------------+", + "+---------------+-------------------------------------------------------------------------------------------------------------+", + "| plan_type | plan |", + "+---------------+-------------------------------------------------------------------------------------------------------------+", + "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", + "| | TableScan: t projection=[a, b] |", + "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] |", + "| | DataSourceExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", + "| | |", + "+---------------+-------------------------------------------------------------------------------------------------------------+", ] ); diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 9a9ec19a7d9b..70e94227cfad 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -613,8 +613,7 @@ async fn test_physical_plan_display_indent() { " CoalesceBatchesExec: target_batch_size=4096", " FilterExec: c12@1 < 10", " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", - " YieldStreamExec child=DataSourceExec", - " DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true", + " DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true", ]; let normalizer = ExplainNormalizer::new(); diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 42c39e3e2554..9474a5f88c92 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -33,12 +33,12 @@ use std::borrow::Cow; use std::sync::Arc; use std::task::{Context, Poll}; -use super::AggregateExec; use crate::filter::batch_filter; - use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use futures::stream::{Stream, StreamExt}; +use super::AggregateExec; + /// stream struct for aggregation without grouping columns pub(crate) struct AggregateStream { stream: BoxStream<'static, Result>, diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 587e0dd538ff..62f541443068 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -448,7 +448,6 @@ impl GroupedHashAggregateStream { let batch_size = context.session_config().batch_size(); let input = agg.input.execute(partition, Arc::clone(&context))?; - let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let timer = baseline_metrics.elapsed_compute().timer(); diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index a3e23b997633..4a6ad5eddfb7 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -57,5 +57,4 @@ physical_plan 08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: 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], file_type=csv, has_header=false +11)--------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index d5a801e44296..04de9153a047 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -89,22 +89,18 @@ physical_plan 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 20)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)----------------------------------------YieldStreamExec child=DataSourceExec -22)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -23)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -25)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)----------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -27)------------------------------------------YieldStreamExec child=DataSourceExec -28)--------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -29)--------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -31)------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -33)----------------------------------YieldStreamExec child=DataSourceExec -34)------------------------------------DataSourceExec: 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], file_type=csv, 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)------------------------YieldStreamExec child=DataSourceExec -39)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +21)----------------------------------------DataSourceExec: 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], file_type=csv, 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 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +26)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +27)--------------------------CoalesceBatchesExec: target_batch_size=8192 +28)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +29)------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +31)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false +32)------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +34)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index 1624589c3c41..a6225daae436 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -89,43 +89,37 @@ physical_plan 14)--------------------------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] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -17)--------------------------------YieldStreamExec child=DataSourceExec -18)----------------------------------DataSourceExec: 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], file_type=csv, 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)----------------------------------YieldStreamExec child=DataSourceExec -23)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -24)--------------------CoalesceBatchesExec: target_batch_size=8192 -25)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -26)------------------------CoalesceBatchesExec: target_batch_size=8192 -27)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -28)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)------------------------------YieldStreamExec child=DataSourceExec -30)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -31)------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)] -32)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -33)----------CoalescePartitionsExec -34)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -35)--------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -37)------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -39)----------------------CoalesceBatchesExec: target_batch_size=8192 -40)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +17)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false +18)----------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +20)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +21)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +22)--------------------CoalesceBatchesExec: target_batch_size=8192 +23)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +24)------------------------CoalesceBatchesExec: target_batch_size=8192 +25)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +26)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +28)------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)] +29)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +30)----------CoalescePartitionsExec +31)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +32)--------------CoalesceBatchesExec: target_batch_size=8192 +33)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +34)------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +36)----------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +38)--------------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +40)------------------------------DataSourceExec: 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], file_type=csv, has_header=false 41)--------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -43)------------------------------YieldStreamExec child=DataSourceExec -44)--------------------------------DataSourceExec: 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], file_type=csv, 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)--------------------------------YieldStreamExec child=DataSourceExec -49)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -50)------------------CoalesceBatchesExec: target_batch_size=8192 -51)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -52)----------------------CoalesceBatchesExec: target_batch_size=8192 -53)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -54)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -55)----------------------------YieldStreamExec child=DataSourceExec -56)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +42)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +43)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +44)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +45)------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +47)----------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +49)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +50)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index ae0b59e92f24..f7344daed8c7 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -72,9 +72,7 @@ physical_plan 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------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 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] -14)--------------------------YieldStreamExec child=DataSourceExec -15)----------------------------DataSourceExec: 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], file_type=csv, has_header=false -16)------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -18)----------------------YieldStreamExec child=DataSourceExec -19)------------------------DataSourceExec: 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], file_type=csv, has_header=false +14)--------------------------DataSourceExec: 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], file_type=csv, has_header=false +15)------------------CoalesceBatchesExec: target_batch_size=8192 +16)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +17)----------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index af290dc30671..96f3bd6edf32 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -67,11 +67,9 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------------------YieldStreamExec child=DataSourceExec -16)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false -17)----------------------CoalesceBatchesExec: target_batch_size=8192 -18)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -19)--------------------------CoalesceBatchesExec: target_batch_size=8192 -20)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -21)------------------------------YieldStreamExec child=DataSourceExec -22)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false +15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false +16)----------------------CoalesceBatchesExec: target_batch_size=8192 +17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +18)--------------------------CoalesceBatchesExec: target_batch_size=8192 +19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +20)------------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index 66ec8c578516..8d8dd68c3d7b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -52,10 +52,8 @@ physical_plan 09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] -12)----------------------YieldStreamExec child=DataSourceExec -13)------------------------DataSourceExec: 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], file_type=csv, has_header=false -14)--------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -16)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------YieldStreamExec child=DataSourceExec -18)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false +12)----------------------DataSourceExec: 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], file_type=csv, 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)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index a7c19c7e78ea..0636a033b25a 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -85,21 +85,18 @@ physical_plan 12)----------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 13)------------------------CoalesceBatchesExec: target_batch_size=8192 14)--------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -15)----------------------------YieldStreamExec child=DataSourceExec -16)------------------------------DataSourceExec: 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], file_type=csv, has_header=false -17)--------CoalesceBatchesExec: target_batch_size=8192 -18)----------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] -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)------------------YieldStreamExec child=DataSourceExec -23)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false -24)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -25)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -26)----------------CoalesceBatchesExec: target_batch_size=8192 -27)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -28)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -29)----------------------CoalesceBatchesExec: target_batch_size=8192 -30)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -31)--------------------------YieldStreamExec child=DataSourceExec -32)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +15)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +16)--------CoalesceBatchesExec: target_batch_size=8192 +17)----------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] +18)------------CoalesceBatchesExec: target_batch_size=8192 +19)--------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +20)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +21)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false +22)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +23)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +24)----------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +26)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +27)----------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +29)--------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index f3a6b9145328..edc452284cf9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -84,19 +84,16 @@ physical_plan 17)--------------------------------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] 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -20)--------------------------------------YieldStreamExec child=DataSourceExec -21)----------------------------------------DataSourceExec: 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], file_type=csv, 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)--------------------------------------------YieldStreamExec child=DataSourceExec -28)----------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=csv, has_header=false -29)--------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -31)------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] -33)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -34)------------------------------------YieldStreamExec child=DataSourceExec -35)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], file_type=csv, has_header=false +20)--------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +21)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +23)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------------------------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) }]) +25)------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +26)--------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=csv, has_header=false +27)--------------------------CoalesceBatchesExec: target_batch_size=8192 +28)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +29)------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] +31)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +32)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index e6afd784083b..51a0d096428c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -61,19 +61,16 @@ physical_plan 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)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: 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], file_type=csv, 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#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] -17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)--------------------------YieldStreamExec child=DataSourceExec -19)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false -20)------------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] -21)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -22)----------------CoalesceBatchesExec: target_batch_size=8192 -23)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -24)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -25)----------------------YieldStreamExec child=DataSourceExec -26)------------------------DataSourceExec: 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], file_type=csv, has_header=false +11)--------------------DataSourceExec: 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], file_type=csv, has_header=false +12)----------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +14)--------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] +16)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false +18)------------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] +19)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +20)----------------CoalesceBatchesExec: target_batch_size=8192 +21)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +22)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +23)----------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 0f23fca7eaa7..55da5371671e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -84,21 +84,17 @@ physical_plan 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)----------------------------------YieldStreamExec child=DataSourceExec -19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], file_type=csv, has_header=false -20)----------------------------CoalesceBatchesExec: target_batch_size=8192 -21)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -22)--------------------------------YieldStreamExec child=DataSourceExec -23)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false -24)--------------------CoalesceBatchesExec: target_batch_size=8192 -25)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -26)------------------------YieldStreamExec child=DataSourceExec -27)--------------------------DataSourceExec: 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], file_type=csv, has_header=false -28)----------------CoalesceBatchesExec: target_batch_size=8192 -29)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] -30)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -31)----------------------CoalesceBatchesExec: target_batch_size=8192 -32)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -33)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -34)----------------------------YieldStreamExec child=DataSourceExec -35)------------------------------DataSourceExec: 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], file_type=csv, has_header=false +18)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], file_type=csv, has_header=false +19)----------------------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +21)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false +22)--------------------CoalesceBatchesExec: target_batch_size=8192 +23)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +24)------------------------DataSourceExec: 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], file_type=csv, has_header=false +25)----------------CoalesceBatchesExec: target_batch_size=8192 +26)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] +27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +28)----------------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +30)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +31)----------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index 85aad1997312..3b15fb3d8e53 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -74,12 +74,10 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------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, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] -11)--------------------YieldStreamExec child=DataSourceExec -12)----------------------DataSourceExec: 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], file_type=csv, 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 p_container@3 IN ([Literal { value: Utf8View("SM CASE") }, Literal { value: Utf8View("SM BOX") }, Literal { value: Utf8View("SM PACK") }, Literal { value: Utf8View("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG") }, Literal { value: Utf8View("MED BOX") }, Literal { value: Utf8View("MED PKG") }, Literal { value: Utf8View("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE") }, Literal { value: Utf8View("LG BOX") }, Literal { value: Utf8View("LG PACK") }, Literal { value: Utf8View("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 -17)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)----------------------YieldStreamExec child=DataSourceExec -19)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=csv, has_header=false +11)--------------------DataSourceExec: 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], file_type=csv, has_header=false +12)------------CoalesceBatchesExec: target_batch_size=8192 +13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +14)----------------CoalesceBatchesExec: target_batch_size=8192 +15)------------------FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE") }, Literal { value: Utf8View("SM BOX") }, Literal { value: Utf8View("SM PACK") }, Literal { value: Utf8View("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG") }, Literal { value: Utf8View("MED BOX") }, Literal { value: Utf8View("MED PKG") }, Literal { value: Utf8View("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE") }, Literal { value: Utf8View("LG BOX") }, Literal { value: Utf8View("LG PACK") }, Literal { value: Utf8View("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 +16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index 18cd8ab1e3b7..b2e0fb0cd1cc 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -126,64 +126,55 @@ physical_plan 25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 26)--------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] 27)----------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)------------------------------------------------------YieldStreamExec child=DataSourceExec -29)--------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false -30)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -32)------------------------------------------------YieldStreamExec child=DataSourceExec -33)--------------------------------------------------DataSourceExec: 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], file_type=csv, 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)------------------------------------------YieldStreamExec child=DataSourceExec -38)--------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -39)--------------------------CoalesceBatchesExec: target_batch_size=8192 -40)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -41)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -42)--------------------------------YieldStreamExec child=DataSourceExec -43)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -44)------------------CoalesceBatchesExec: target_batch_size=8192 -45)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -46)----------------------CoalesceBatchesExec: target_batch_size=8192 -47)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -48)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -49)----------------------------YieldStreamExec child=DataSourceExec -50)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false -51)----------CoalesceBatchesExec: target_batch_size=8192 -52)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -53)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -54)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -55)------------------CoalesceBatchesExec: target_batch_size=8192 -56)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -57)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -58)------------------------CoalesceBatchesExec: target_batch_size=8192 -59)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -60)----------------------------CoalesceBatchesExec: target_batch_size=8192 -61)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -62)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -63)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -64)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -65)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -66)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -67)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -68)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -69)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -70)------------------------------------------------YieldStreamExec child=DataSourceExec -71)--------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -72)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -73)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -74)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -75)--------------------------------------------------YieldStreamExec child=DataSourceExec -76)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -77)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -78)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -79)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -80)------------------------------------------YieldStreamExec child=DataSourceExec -81)--------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -82)----------------------------CoalesceBatchesExec: target_batch_size=8192 -83)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -84)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -85)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -86)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -87)--------------------------------------YieldStreamExec child=DataSourceExec -88)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +28)------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false +29)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +31)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +32)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +34)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +36)--------------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +38)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +39)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false +40)------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +42)----------------------CoalesceBatchesExec: target_batch_size=8192 +43)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +44)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +45)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +46)----------CoalesceBatchesExec: target_batch_size=8192 +47)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +48)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +49)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +50)------------------CoalesceBatchesExec: target_batch_size=8192 +51)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +52)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +53)------------------------CoalesceBatchesExec: target_batch_size=8192 +54)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +55)----------------------------CoalesceBatchesExec: target_batch_size=8192 +56)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +57)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +59)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +60)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +61)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +62)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +63)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +64)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +65)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +66)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +67)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +68)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +69)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +70)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +71)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +72)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +73)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false +74)----------------------------CoalesceBatchesExec: target_batch_size=8192 +75)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +76)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +77)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +78)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +79)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 8ff8388687fb..0b994de411ea 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -92,40 +92,35 @@ physical_plan 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)----------------------YieldStreamExec child=DataSourceExec -13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=csv, has_header=false -14)----------------CoalesceBatchesExec: target_batch_size=8192 -15)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -16)--------------------CoalesceBatchesExec: target_batch_size=8192 -17)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] -18)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)--------------------------YieldStreamExec child=DataSourceExec -20)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -21)--------CoalesceBatchesExec: target_batch_size=8192 -22)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -23)------------CoalesceBatchesExec: target_batch_size=8192 -24)--------------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] -25)----------------CoalesceBatchesExec: target_batch_size=8192 -26)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -27)--------------------CoalesceBatchesExec: target_batch_size=8192 -28)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -29)------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -31)----------------------------YieldStreamExec child=DataSourceExec -32)------------------------------DataSourceExec: 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], file_type=csv, has_header=false -33)------------------------CoalesceBatchesExec: target_batch_size=8192 -34)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -35)----------------------------CoalesceBatchesExec: target_batch_size=8192 -36)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] -37)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -38)----------------------------------YieldStreamExec child=DataSourceExec -39)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false -40)----------------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] -41)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -42)--------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -44)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -45)--------------------------CoalesceBatchesExec: target_batch_size=8192 -46)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] -47)------------------------------YieldStreamExec child=DataSourceExec -48)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=csv, has_header=false +13)----------------CoalesceBatchesExec: target_batch_size=8192 +14)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +15)--------------------CoalesceBatchesExec: target_batch_size=8192 +16)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] +17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +19)--------CoalesceBatchesExec: target_batch_size=8192 +20)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +21)------------CoalesceBatchesExec: target_batch_size=8192 +22)--------------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] +23)----------------CoalesceBatchesExec: target_batch_size=8192 +24)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +25)--------------------CoalesceBatchesExec: target_batch_size=8192 +26)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +27)------------------------CoalesceBatchesExec: target_batch_size=8192 +28)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +29)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +30)------------------------CoalesceBatchesExec: target_batch_size=8192 +31)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +32)----------------------------CoalesceBatchesExec: target_batch_size=8192 +33)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] +34)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false +36)----------------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] +37)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +38)--------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +40)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +41)--------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] +43)------------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index fe7ae6f6f03b..e52171524007 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -116,34 +116,28 @@ physical_plan 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)----------------------------------------------------YieldStreamExec child=DataSourceExec -28)------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=csv, has_header=false -29)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -31)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -33)------------------------------------------------------YieldStreamExec child=DataSourceExec -34)--------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -35)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -37)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] -39)----------------------------------------------YieldStreamExec child=DataSourceExec -40)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -41)------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -43)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -44)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] -45)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -46)----------------------------------------YieldStreamExec child=DataSourceExec -47)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -48)----------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -50)--------------------------YieldStreamExec child=DataSourceExec -51)----------------------------DataSourceExec: 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], file_type=csv, has_header=false -52)------------------CoalesceBatchesExec: target_batch_size=8192 -53)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -54)----------------------CoalesceBatchesExec: target_batch_size=8192 -55)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -56)--------------------------YieldStreamExec child=DataSourceExec -57)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +27)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=csv, has_header=false +28)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +32)------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +33)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] +37)----------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +38)------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +40)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] +42)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +43)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +44)----------------------CoalesceBatchesExec: target_batch_size=8192 +45)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +46)--------------------------DataSourceExec: 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], file_type=csv, has_header=false +47)------------------CoalesceBatchesExec: target_batch_size=8192 +48)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +49)----------------------CoalesceBatchesExec: target_batch_size=8192 +50)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +51)--------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index 71b1f0554da1..828bf967d8f4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -92,17 +92,14 @@ physical_plan 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 17)--------------------------------FilterExec: substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13") }, Literal { value: Utf8View("31") }, Literal { value: Utf8View("23") }, Literal { value: Utf8View("29") }, Literal { value: Utf8View("30") }, Literal { value: Utf8View("18") }, Literal { value: Utf8View("17") }]) 18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------------------YieldStreamExec child=DataSourceExec -20)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false -21)--------------------------CoalesceBatchesExec: target_batch_size=8192 -22)----------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -23)------------------------------YieldStreamExec child=DataSourceExec -24)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false -25)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] -26)----------------------CoalescePartitionsExec -27)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -28)--------------------------CoalesceBatchesExec: target_batch_size=8192 -29)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13") }, Literal { value: Utf8View("31") }, Literal { value: Utf8View("23") }, Literal { value: Utf8View("29") }, Literal { value: Utf8View("30") }, Literal { value: Utf8View("18") }, Literal { value: Utf8View("17") }]), projection=[c_acctbal@1] -30)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -31)--------------------------------YieldStreamExec child=DataSourceExec -32)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false +19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false +20)--------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 +22)------------------------------DataSourceExec: 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], file_type=csv, has_header=false +23)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] +24)----------------------CoalescePartitionsExec +25)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] +26)--------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13") }, Literal { value: Utf8View("31") }, Literal { value: Utf8View("23") }, Literal { value: Utf8View("29") }, Literal { value: Utf8View("30") }, Literal { value: Utf8View("18") }, Literal { value: Utf8View("17") }]), projection=[c_acctbal@1] +28)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index 99aa0cc70e9e..d982ec32e954 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -76,17 +76,14 @@ physical_plan 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 17)--------------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] 18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------------------YieldStreamExec child=DataSourceExec -20)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], file_type=csv, has_header=false -21)--------------------------CoalesceBatchesExec: target_batch_size=8192 -22)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -23)------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 -25)----------------------------------YieldStreamExec child=DataSourceExec -26)------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -27)------------------CoalesceBatchesExec: target_batch_size=8192 -28)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -29)----------------------CoalesceBatchesExec: target_batch_size=8192 -30)------------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -31)--------------------------YieldStreamExec child=DataSourceExec -32)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], file_type=csv, has_header=false +20)--------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +22)------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 +24)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false +25)------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +27)----------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +29)--------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index d8194c838409..f7de3cd3c967 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -66,11 +66,9 @@ physical_plan 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] -14)--------------------------YieldStreamExec child=DataSourceExec -15)----------------------------DataSourceExec: 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], file_type=csv, has_header=false -16)------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -18)----------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] -20)--------------------------YieldStreamExec child=DataSourceExec -21)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +14)--------------------------DataSourceExec: 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], file_type=csv, has_header=false +15)------------------CoalesceBatchesExec: target_batch_size=8192 +16)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +17)----------------------CoalesceBatchesExec: target_batch_size=8192 +18)------------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] +19)--------------------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index 54c1068e2697..15636056b871 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -95,32 +95,26 @@ physical_plan 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)--------------------------------------------------------YieldStreamExec child=DataSourceExec -30)----------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -31)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -33)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -35)----------------------------------------------------------YieldStreamExec child=DataSourceExec -36)------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -37)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -39)----------------------------------------------YieldStreamExec child=DataSourceExec -40)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -41)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 -43)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -44)----------------------------------------YieldStreamExec child=DataSourceExec -45)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -46)--------------------------CoalesceBatchesExec: target_batch_size=8192 -47)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -48)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -49)--------------------------------YieldStreamExec child=DataSourceExec -50)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -51)------------------CoalesceBatchesExec: target_batch_size=8192 -52)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -53)----------------------CoalesceBatchesExec: target_batch_size=8192 -54)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] -55)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -56)----------------------------YieldStreamExec child=DataSourceExec -57)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +29)--------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false +30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +32)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +34)----------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +37)----------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +38)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 +40)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +41)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +42)--------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +45)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false +46)------------------CoalesceBatchesExec: target_batch_size=8192 +47)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +48)----------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +51)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part index 1d026866ef91..b1e5d2869a8c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part @@ -40,5 +40,4 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] -07)------------YieldStreamExec child=DataSourceExec -08)--------------DataSourceExec: 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], file_type=csv, has_header=false +07)------------DataSourceExec: 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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index 8b0c733f701c..291d56e43f2d 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -113,34 +113,28 @@ physical_plan 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)----------------------------------------------------------YieldStreamExec child=DataSourceExec -31)------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -32)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -34)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)----------------------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 -36)------------------------------------------------------------YieldStreamExec child=DataSourceExec -37)--------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -38)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -40)------------------------------------------------YieldStreamExec child=DataSourceExec -41)--------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -42)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -43)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -44)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)------------------------------------------YieldStreamExec child=DataSourceExec -46)--------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -47)----------------------------CoalesceBatchesExec: target_batch_size=8192 -48)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -49)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -50)----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY -51)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -52)--------------------------------------YieldStreamExec child=DataSourceExec -53)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -54)--------------------CoalesceBatchesExec: target_batch_size=8192 -55)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -56)------------------------CoalesceBatchesExec: target_batch_size=8192 -57)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -58)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -59)------------------------------YieldStreamExec child=DataSourceExec -60)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +30)----------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, 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 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 +35)------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +38)------------------------------------------------DataSourceExec: 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], file_type=csv, 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)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, 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)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, 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)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 80a53ad3dbff..50171c528db6 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -129,42 +129,34 @@ physical_plan 37)------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 38)--------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] 39)----------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -40)------------------------------------------------------------------------------YieldStreamExec child=DataSourceExec -41)--------------------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false -42)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -44)------------------------------------------------------------------------YieldStreamExec child=DataSourceExec -45)--------------------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -46)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -48)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -49)------------------------------------------------------------------YieldStreamExec child=DataSourceExec -50)--------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -51)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -52)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -53)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -54)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -55)------------------------------------------------------------YieldStreamExec child=DataSourceExec -56)--------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -57)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -59)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -60)--------------------------------------------------YieldStreamExec child=DataSourceExec -61)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, 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)------------------------------------------YieldStreamExec child=DataSourceExec -66)--------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -67)----------------------------CoalesceBatchesExec: target_batch_size=8192 -68)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -69)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -70)----------------------------------YieldStreamExec child=DataSourceExec -71)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -72)--------------------CoalesceBatchesExec: target_batch_size=8192 -73)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -74)------------------------CoalesceBatchesExec: target_batch_size=8192 -75)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] -76)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -77)------------------------------YieldStreamExec child=DataSourceExec -78)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +40)------------------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false +41)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +43)------------------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +44)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +45)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +46)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +47)------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +48)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +50)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +51)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +52)------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +53)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +54)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +55)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +56)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false +57)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +59)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +60)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false +61)----------------------------CoalesceBatchesExec: target_batch_size=8192 +62)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +63)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +64)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +65)--------------------CoalesceBatchesExec: target_batch_size=8192 +66)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +67)------------------------CoalesceBatchesExec: target_batch_size=8192 +68)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] +69)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +70)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 4290d93f57dc..3b31c1bc2e8e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -106,27 +106,21 @@ physical_plan 29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 30)----------------------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] 31)------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)--------------------------------------------------------------YieldStreamExec child=DataSourceExec -33)----------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false -34)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -36)--------------------------------------------------------YieldStreamExec child=DataSourceExec -37)----------------------------------------------------------DataSourceExec: 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], file_type=csv, 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)--------------------------------------------------YieldStreamExec child=DataSourceExec -42)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -43)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -44)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -45)----------------------------------------YieldStreamExec child=DataSourceExec -46)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -47)----------------------------CoalesceBatchesExec: target_batch_size=8192 -48)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -49)--------------------------------YieldStreamExec child=DataSourceExec -50)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false -51)--------------------CoalesceBatchesExec: target_batch_size=8192 -52)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -53)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -54)--------------------------YieldStreamExec child=DataSourceExec -55)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +32)--------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false +33)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +35)--------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +38)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +39)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +40)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +42)----------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +43)----------------------------CoalesceBatchesExec: target_batch_size=8192 +44)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +45)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false +46)--------------------CoalesceBatchesExec: target_batch_size=8192 +47)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +48)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +49)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false From 485d55a806501136dd1e88034fb1595ff279ebe4 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 4 Jun 2025 22:09:16 +0800 Subject: [PATCH 34/74] fix --- datafusion/core/tests/dataframe/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index aa36de1e555f..089ff8808134 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -907,7 +907,7 @@ async fn window_using_aggregates() -> Result<()> { vec![col("c3")], ); - Expr::from(w) + Expr::WindowFunction(w) .null_treatment(NullTreatment::IgnoreNulls) .order_by(vec![col("c2").sort(true, true), col("c3").sort(true, true)]) .window_frame(WindowFrame::new_bounds( From e31a99748d7ff9793cc3451cf420b23a696c0389 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 4 Jun 2025 22:13:58 +0800 Subject: [PATCH 35/74] Update doc --- docs/source/user-guide/configs.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index a794241dfc4f..61f7195d3cbe 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -119,6 +119,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | | datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | | datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | +| datafusion.optimizer.enable_add_yield_for_pipeline_break | false | When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. Default: false (disabled). | | datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | | datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | | datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | From b65ab6082d759e4b20c6a50baae4bb3fb90daae4 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 4 Jun 2025 22:33:09 +0800 Subject: [PATCH 36/74] Address comments --- .../physical_optimizer/enforce_distribution.rs | 16 +++++++--------- datafusion/physical-plan/src/yield_stream.rs | 6 +++--- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 638156838798..e6d102fbec36 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3523,23 +3523,21 @@ async fn test_distribute_sort_parquet() -> Result<()> { /// Ensures that `DataSourceExec` has been repartitioned into `target_partitions` memtable groups #[tokio::test] async fn test_distribute_sort_memtable() -> Result<()> { - let mut test_config: TestConfig = + let test_config: TestConfig = TestConfig::default().with_prefer_repartition_file_scans(1000); assert!( test_config.config.optimizer.repartition_file_scans, "should enable scans to be repartitioned" ); - // Enable add yield for pipeline break testing. - test_config - .config - .optimizer - .enable_add_yield_for_pipeline_break = true; - let mem_table = create_memtable()?; - let session_config = SessionConfig::new() + let mut session_config = SessionConfig::new() .with_repartition_file_min_size(1000) .with_target_partitions(3); + + // Enable add yield for pipeline break testing. + session_config.options_mut().optimizer.enable_add_yield_for_pipeline_break = true; + let ctx = SessionContext::new_with_config(session_config); ctx.register_table("users", Arc::new(mem_table))?; @@ -3550,7 +3548,7 @@ async fn test_distribute_sort_memtable() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [id@0 ASC NULLS LAST]", " SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true]", - " YieldStreamExec child=DataSourceExec", + " YieldStreamExec frequency=64", " DataSourceExec: partitions=3, partition_sizes=[34, 33, 33]", ]; plans_matches_expected!(expected, physical_plan); diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index 537547d3744c..868554c1cf92 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -93,14 +93,14 @@ impl RecordBatchStream for YieldStream { #[derive(Debug)] pub struct YieldStreamExec { child: Arc, - + frequency: usize, properties: PlanProperties, } impl YieldStreamExec { pub fn new(child: Arc) -> Self { let properties = child.properties().clone(); - Self { child, properties } + Self { child, properties, frequency: YIELD_BATCHES } } } @@ -110,7 +110,7 @@ impl DisplayAs for YieldStreamExec { _t: DisplayFormatType, f: &mut std::fmt::Formatter<'_>, ) -> std::fmt::Result { - write!(f, "YieldStreamExec child={}", self.child.name()) + write!(f, "YieldStreamExec frequency={}", self.frequency) } } From 1bf2ad3a93313f527b3a0d8201b3ecef77b76249 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 4 Jun 2025 22:35:23 +0800 Subject: [PATCH 37/74] fix fmt --- .../core/tests/physical_optimizer/enforce_distribution.rs | 5 ++++- datafusion/physical-plan/src/yield_stream.rs | 6 +++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index e6d102fbec36..c06d63179fde 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3536,7 +3536,10 @@ async fn test_distribute_sort_memtable() -> Result<()> { .with_target_partitions(3); // Enable add yield for pipeline break testing. - session_config.options_mut().optimizer.enable_add_yield_for_pipeline_break = true; + session_config + .options_mut() + .optimizer + .enable_add_yield_for_pipeline_break = true; let ctx = SessionContext::new_with_config(session_config); ctx.register_table("users", Arc::new(mem_table))?; diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index 868554c1cf92..9d17f282df11 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -100,7 +100,11 @@ pub struct YieldStreamExec { impl YieldStreamExec { pub fn new(child: Arc) -> Self { let properties = child.properties().clone(); - Self { child, properties, frequency: YIELD_BATCHES } + Self { + child, + properties, + frequency: YIELD_BATCHES, + } } } From 8ab64b34b4db0200310b438b340878cd68d50796 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 4 Jun 2025 23:18:17 +0800 Subject: [PATCH 38/74] Support config for yield frequency --- datafusion/common/src/config.rs | 7 +++ .../src/wrap_leaves_cancellation.rs | 19 ++++++-- datafusion/physical-plan/src/yield_stream.rs | 46 +++++++++++++------ datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 19 ++++++++ datafusion/proto/src/generated/prost.rs | 2 + datafusion/proto/src/physical_plan/mod.rs | 6 ++- .../test_files/information_schema.slt | 2 + docs/source/user-guide/configs.md | 1 + 9 files changed, 83 insertions(+), 20 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index bc449dc93271..256e173b6d92 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -728,6 +728,13 @@ config_namespace! { /// other tasks while waiting. /// Default: false (disabled). pub enable_add_yield_for_pipeline_break: bool, default = false + + /// Yield frequency in batches, it represents how many batches to process before yielding + /// to the Tokio scheduler. The default value is 64, which means that after processing + /// 64 batches, the execution will yield control back to the Tokio scheduler. + /// This setting is only effective when `enable_add_yield_for_pipeline_break` is set to true. + /// This value should be greater than 0. + pub yield_frequency_for_pipeline_break: usize, default = 64 } } diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index d8829e3987c6..1a3d9e7d0d03 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -40,10 +40,11 @@ impl WrapLeaves { /// and stop recursing further under that branch (TreeNodeRecursion::Jump). fn wrap_leaves( plan: Arc, + yield_frequency: usize, ) -> Result>> { if plan.children().is_empty() { // Leaf: wrap it in YieldStreamExec, and do not descend further - let wrapped = Arc::new(YieldStreamExec::new(plan)); + let wrapped = Arc::new(YieldStreamExec::new(plan, yield_frequency)); Ok(Transformed::new( wrapped, /* changed */ true, @@ -63,11 +64,16 @@ impl WrapLeaves { /// this subtree (we’ve already wrapped its leaves). fn wrap_leaves_of_pipeline_breakers( plan: Arc, + yield_frequency: usize, ) -> Result>> { let is_pipeline_breaker = plan.properties().emission_type == EmissionType::Final; if is_pipeline_breaker { // Transform all leaf descendants of this node by calling wrap_leaves - let mut transformed = plan.transform_down(Self::wrap_leaves)?; + let mut transformed = + plan.transform_down(|child_plan: Arc| { + Self::wrap_leaves(child_plan, yield_frequency) + })?; + // Once we’ve handled the leaves of this subtree, we skip deeper recursion transformed.tnr = TreeNodeRecursion::Jump; Ok(transformed) @@ -101,10 +107,15 @@ impl PhysicalOptimizerRule for WrapLeaves { config: &ConfigOptions, ) -> Result> { if config.optimizer.enable_add_yield_for_pipeline_break { + let yield_frequency = config.optimizer.yield_frequency_for_pipeline_break; + // We run a top‐level transform_down: for every node, call wrap_leaves_of_pipeline_breakers. // If a node is a pipeline breaker, we then wrap all of its leaf children in YieldStreamExec. - plan.transform_down(Self::wrap_leaves_of_pipeline_breakers) - .map(|t| t.data) + let new_plan = plan.transform_down(|node: Arc| { + Self::wrap_leaves_of_pipeline_breakers(node, yield_frequency) + })?; + + Ok(new_plan.data) } else { Ok(plan) } diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index 9d17f282df11..b113fd63aa21 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -32,24 +32,21 @@ use datafusion_common::{Result, Statistics}; use datafusion_execution::TaskContext; use futures::{Stream, StreamExt}; -/// Number of batches to yield before voluntarily returning Pending. -/// This allows long-running operators to periodically yield control -/// back to the executor (e.g., to handle cancellation). -const YIELD_BATCHES: usize = 64; - -/// A stream that yields batches of data, yielding control back to the executor every `YIELD_BATCHES` batches +/// A stream that yields batches of data, yielding control back to the executor every `frequency` batches /// /// This can be useful to allow operators that might not yield to check for cancellation pub struct YieldStream { inner: SendableRecordBatchStream, batches_processed: usize, + frequency: usize, } impl YieldStream { - pub fn new(inner: SendableRecordBatchStream) -> Self { + pub fn new(inner: SendableRecordBatchStream, frequency: usize) -> Self { Self { inner, batches_processed: 0, + frequency, } } } @@ -62,7 +59,7 @@ impl Stream for YieldStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - if self.batches_processed >= YIELD_BATCHES { + if self.batches_processed >= self.frequency { self.batches_processed = 0; cx.waker().wake_by_ref(); return Poll::Pending; @@ -98,12 +95,12 @@ pub struct YieldStreamExec { } impl YieldStreamExec { - pub fn new(child: Arc) -> Self { + pub fn new(child: Arc, frequency: usize) -> Self { let properties = child.properties().clone(); Self { child, properties, - frequency: YIELD_BATCHES, + frequency, } } } @@ -124,6 +121,12 @@ impl YieldStreamExec { } } +impl YieldStreamExec { + pub fn get_yield_frequency(&self) -> usize { + self.frequency + } +} + impl ExecutionPlan for YieldStreamExec { fn name(&self) -> &str { "YieldStreamExec" @@ -155,7 +158,10 @@ impl ExecutionPlan for YieldStreamExec { )); } // Use Arc::clone on children[0] rather than calling clone() directly - Ok(Arc::new(YieldStreamExec::new(Arc::clone(&children[0])))) + Ok(Arc::new(YieldStreamExec::new( + Arc::clone(&children[0]), + self.frequency, + ))) } fn execute( @@ -164,7 +170,7 @@ impl ExecutionPlan for YieldStreamExec { task_ctx: Arc, ) -> Result { let child_stream = self.child.execute(partition, Arc::clone(&task_ctx))?; - let yield_stream = YieldStream::new(child_stream); + let yield_stream = YieldStream::new(child_stream, self.frequency); Ok(Box::pin(yield_stream)) } @@ -197,6 +203,10 @@ mod tests { use arrow_schema::SchemaRef; use futures::{stream, StreamExt}; + // Frequency testing: + // Number of batches to yield before yielding control back to the executor + const YIELD_BATCHES: usize = 64; + /// Helper: construct a SendableRecordBatchStream containing `n` empty batches fn make_empty_batches(n: usize) -> SendableRecordBatchStream { let schema: SchemaRef = Arc::new(Schema::empty()); @@ -214,7 +224,9 @@ mod tests { async fn yield_less_than_threshold() -> Result<()> { let count = YIELD_BATCHES - 10; let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).collect::>().await; + let out: Vec<_> = YieldStream::new(inner, YIELD_BATCHES) + .collect::>() + .await; assert_eq!(out.len(), count); Ok(()) } @@ -223,7 +235,9 @@ mod tests { async fn yield_equal_to_threshold() -> Result<()> { let count = YIELD_BATCHES; let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).collect::>().await; + let out: Vec<_> = YieldStream::new(inner, YIELD_BATCHES) + .collect::>() + .await; assert_eq!(out.len(), count); Ok(()) } @@ -232,7 +246,9 @@ mod tests { async fn yield_more_than_threshold() -> Result<()> { let count = YIELD_BATCHES + 20; let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner).collect::>().await; + let out: Vec<_> = YieldStream::new(inner, YIELD_BATCHES) + .collect::>() + .await; assert_eq!(out.len(), count); Ok(()) } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 460a1ae81497..76cd8c9118a2 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1036,6 +1036,7 @@ message AvroScanExecNode { message YieldStreamExecNode { PhysicalPlanNode input = 1; + uint32 frequency = 2; } enum PartitionMode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 64ca87b895c4..8a62be84ec8f 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -22690,10 +22690,16 @@ impl serde::Serialize for YieldStreamExecNode { if self.input.is_some() { len += 1; } + if self.frequency != 0 { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.YieldStreamExecNode", len)?; if let Some(v) = self.input.as_ref() { struct_ser.serialize_field("input", v)?; } + if self.frequency != 0 { + struct_ser.serialize_field("frequency", &self.frequency)?; + } struct_ser.end() } } @@ -22705,11 +22711,13 @@ impl<'de> serde::Deserialize<'de> for YieldStreamExecNode { { const FIELDS: &[&str] = &[ "input", + "frequency", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { Input, + Frequency, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -22732,6 +22740,7 @@ impl<'de> serde::Deserialize<'de> for YieldStreamExecNode { { match value { "input" => Ok(GeneratedField::Input), + "frequency" => Ok(GeneratedField::Frequency), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -22752,6 +22761,7 @@ impl<'de> serde::Deserialize<'de> for YieldStreamExecNode { V: serde::de::MapAccess<'de>, { let mut input__ = None; + let mut frequency__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { @@ -22760,10 +22770,19 @@ impl<'de> serde::Deserialize<'de> for YieldStreamExecNode { } input__ = map_.next_value()?; } + GeneratedField::Frequency => { + if frequency__.is_some() { + return Err(serde::de::Error::duplicate_field("frequency")); + } + frequency__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } } } Ok(YieldStreamExecNode { input: input__, + frequency: frequency__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index f289d42c69c8..3e3a04051f5d 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1577,6 +1577,8 @@ pub struct AvroScanExecNode { pub struct YieldStreamExecNode { #[prost(message, optional, boxed, tag = "1")] pub input: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(uint32, tag = "2")] + pub frequency: u32, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct HashJoinExecNode { diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 020baa09c195..f701b28d6d3a 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1794,7 +1794,8 @@ impl protobuf::PhysicalPlanNode { ) -> Result> { let input = into_physical_plan(&field_stream.input, registry, runtime, extension_codec)?; - Ok(Arc::new(YieldStreamExec::new(input))) + let frequency = field_stream.frequency as usize; + Ok(Arc::new(YieldStreamExec::new(input, frequency))) } fn try_from_explain_exec( @@ -2786,10 +2787,13 @@ impl protobuf::PhysicalPlanNode { extension_codec, )?; + let frequency = exec.get_yield_frequency(); + Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::YieldStream(Box::new( protobuf::YieldStreamExecNode { input: Some(Box::new(input)), + frequency: frequency as u32, }, ))), }) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index ed0e675f7715..b6a32c327f66 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -304,6 +304,7 @@ datafusion.optimizer.repartition_sorts true datafusion.optimizer.repartition_windows true datafusion.optimizer.skip_failed_rules false datafusion.optimizer.top_down_join_key_reordering true +datafusion.optimizer.yield_frequency_for_pipeline_break 64 datafusion.sql_parser.collect_spans false datafusion.sql_parser.dialect generic datafusion.sql_parser.enable_ident_normalization true @@ -415,6 +416,7 @@ datafusion.optimizer.repartition_sorts true Should DataFusion execute sorts in a datafusion.optimizer.repartition_windows true Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level datafusion.optimizer.skip_failed_rules false When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail datafusion.optimizer.top_down_join_key_reordering true When set to true, the physical plan optimizer will run a top down process to reorder the join keys +datafusion.optimizer.yield_frequency_for_pipeline_break 64 Yield frequency in batches, it represents how many batches to process before yielding to the Tokio scheduler. The default value is 64, which means that after processing 64 batches, the execution will yield control back to the Tokio scheduler. This setting is only effective when `enable_add_yield_for_pipeline_break` is set to true. This value should be greater than 0. datafusion.sql_parser.collect_spans false When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. datafusion.sql_parser.dialect generic Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. datafusion.sql_parser.enable_ident_normalization true When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 61f7195d3cbe..b864886c89bf 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -120,6 +120,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | | datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | | datafusion.optimizer.enable_add_yield_for_pipeline_break | false | When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. Default: false (disabled). | +| datafusion.optimizer.yield_frequency_for_pipeline_break | 64 | Yield frequency in batches, it represents how many batches to process before yielding to the Tokio scheduler. The default value is 64, which means that after processing 64 batches, the execution will yield control back to the Tokio scheduler. This setting is only effective when `enable_add_yield_for_pipeline_break` is set to true. This value should be greater than 0. | | datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | | datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | | datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | From c7774a78b809031319c679ad62365d57d575d58d Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 5 Jun 2025 12:37:01 +0800 Subject: [PATCH 39/74] add built-in yield support --- datafusion/common/src/config.rs | 4 +- .../enforce_distribution.rs | 11 +--- datafusion/datasource/src/source.rs | 59 ++++++++++++++++++- .../src/wrap_leaves_cancellation.rs | 24 +++++--- .../physical-plan/src/execution_plan.rs | 31 +++++++++- datafusion/sqllogictest/test_files/array.slt | 18 ++++-- datafusion/sqllogictest/test_files/cte.slt | 14 +++-- .../sqllogictest/test_files/explain_tree.slt | 18 ++++-- .../test_files/information_schema.slt | 4 +- datafusion/sqllogictest/test_files/joins.slt | 3 +- datafusion/sqllogictest/test_files/union.slt | 22 ++++--- datafusion/sqllogictest/test_files/window.slt | 36 ++++++----- docs/source/user-guide/configs.md | 2 +- 13 files changed, 180 insertions(+), 66 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 256e173b6d92..d599e475f5c0 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -726,8 +726,8 @@ config_namespace! { /// When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline /// that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to /// other tasks while waiting. - /// Default: false (disabled). - pub enable_add_yield_for_pipeline_break: bool, default = false + /// Default: true (enabled). + pub enable_add_yield_for_pipeline_break: bool, default = true /// Yield frequency in batches, it represents how many batches to process before yielding /// to the Tokio scheduler. The default value is 64, which means that after processing diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index c06d63179fde..3a021c412944 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3531,16 +3531,10 @@ async fn test_distribute_sort_memtable() -> Result<()> { ); let mem_table = create_memtable()?; - let mut session_config = SessionConfig::new() + let session_config = SessionConfig::new() .with_repartition_file_min_size(1000) .with_target_partitions(3); - // Enable add yield for pipeline break testing. - session_config - .options_mut() - .optimizer - .enable_add_yield_for_pipeline_break = true; - let ctx = SessionContext::new_with_config(session_config); ctx.register_table("users", Arc::new(mem_table))?; @@ -3551,8 +3545,7 @@ async fn test_distribute_sort_memtable() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [id@0 ASC NULLS LAST]", " SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true]", - " YieldStreamExec frequency=64", - " DataSourceExec: partitions=3, partition_sizes=[34, 33, 33]", + " DataSourceExec: partitions=3, partition_sizes=[34, 33, 33]", ]; plans_matches_expected!(expected, physical_plan); diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 30ecc38709f4..d973882946dd 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -38,6 +38,7 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::filter_pushdown::{ ChildPushdownResult, FilterPushdownPropagation, }; +use datafusion_physical_plan::yield_stream::YieldStream; /// A source of data, typically a list of files or memory /// @@ -179,12 +180,17 @@ pub trait DataSource: Send + Sync + Debug { /// the [`FileSource`] trait. /// /// [`FileSource`]: crate::file::FileSource +/// We now add a `cooperative` flag to +/// let it optionally yield back to the runtime periodically. +/// Default is `true`, meaning it will yield back to the runtime for cooperative scheduling. #[derive(Clone, Debug)] pub struct DataSourceExec { /// The source of the data -- for example, `FileScanConfig` or `MemorySourceConfig` data_source: Arc, /// Cached plan properties such as sort order cache: PlanProperties, + /// Indicates whether to enable cooperative yielding mode. + cooperative: bool, } impl DisplayAs for DataSourceExec { @@ -256,7 +262,39 @@ impl ExecutionPlan for DataSourceExec { partition: usize, context: Arc, ) -> Result { - self.data_source.open(partition, context) + // 1. Get the “base” stream exactly as before, without yielding. + let stream = self.data_source.open(partition, Arc::clone(&context)); + + // 2. If cooperative == false, return base_stream immediately. + if !self.cooperative { + return stream; + } + + let frequency = context + .session_config() + .options() + .optimizer + .yield_frequency_for_pipeline_break; + + // 3. If cooperative == true, wrap the stream into a YieldStream. + let yielding_stream = YieldStream::new(stream?, frequency); + Ok(Box::pin(yielding_stream)) + } + + /// Override: this operator *does* support cooperative yielding when `cooperative == true`. + fn yields_cooperatively(&self) -> bool { + self.cooperative + } + + /// If `cooperative == true`, return `Some(self.clone())` so the optimizer knows + /// we can replace a plain DataSourceExec with this same node (it already yields). + /// Otherwise, return None. + fn with_cooperative_yields(self: Arc) -> Option> { + if self.cooperative { + Some(self) + } else { + None + } } fn metrics(&self) -> Option { @@ -289,7 +327,11 @@ impl ExecutionPlan for DataSourceExec { let data_source = self.data_source.with_fetch(limit)?; let cache = self.cache.clone(); - Some(Arc::new(Self { data_source, cache })) + Some(Arc::new(Self { + data_source, + cache, + cooperative: self.cooperative, + })) } fn fetch(&self) -> Option { @@ -337,9 +379,14 @@ impl DataSourceExec { Arc::new(Self::new(Arc::new(data_source))) } + // Default constructor for `DataSourceExec`, setting the `cooperative` flag to `true`. pub fn new(data_source: Arc) -> Self { let cache = Self::compute_properties(Arc::clone(&data_source)); - Self { data_source, cache } + Self { + data_source, + cache, + cooperative: true, + } } /// Return the source object @@ -365,6 +412,12 @@ impl DataSourceExec { self } + /// Assign yielding mode + pub fn with_cooperative(mut self, cooperative: bool) -> Self { + self.cooperative = cooperative; + self + } + fn compute_properties(data_source: Arc) -> PlanProperties { PlanProperties::new( data_source.eq_properties(), diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index 1a3d9e7d0d03..450429eeba1e 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -43,13 +43,23 @@ impl WrapLeaves { yield_frequency: usize, ) -> Result>> { if plan.children().is_empty() { - // Leaf: wrap it in YieldStreamExec, and do not descend further - let wrapped = Arc::new(YieldStreamExec::new(plan, yield_frequency)); - Ok(Transformed::new( - wrapped, - /* changed */ true, - TreeNodeRecursion::Jump, - )) + // If the leaf node already has a built-in yielding variant: + if let Some(coop_variant) = Arc::clone(&plan).with_cooperative_yields() { + // Replace it with the built-in yielding version. + Ok(Transformed::new( + coop_variant, + /* changed = */ true, + TreeNodeRecursion::Jump, + )) + } else { + // Otherwise, wrap in a YieldStreamExec. + let wrapped = Arc::new(YieldStreamExec::new(plan, yield_frequency)); + Ok(Transformed::new( + wrapped, + /* changed = */ true, + TreeNodeRecursion::Jump, + )) + } } else { // Not a leaf: leave unchanged and keep recursing Ok(Transformed::no(plan)) diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index b81b3c8beeac..cf60260d08d4 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -75,7 +75,19 @@ use futures::stream::{StreamExt, TryStreamExt}; /// [`execute`]: ExecutionPlan::execute /// [`required_input_distribution`]: ExecutionPlan::required_input_distribution /// [`required_input_ordering`]: ExecutionPlan::required_input_ordering -pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { +/// The core trait for a physical execution plan node. Every operator +/// implements this trait. We have extended it by adding two new methods +/// for “cooperative yielding” support: +/// +/// 1. `yields_cooperatively()` indicates whether this operator already +/// supports async/yield behavior internally (default: false). +/// +/// 2. `with_cooperative_yields(self: Arc)` returns an alternate +/// plan node that has built-in yielding; if not available, returns None. +/// +/// Because `with_cooperative_yields` moves `Arc` into `Arc`, +/// we must ensure `Self: 'static`. Therefore, we add `+ 'static` here. +pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync + 'static { /// Short name for the ExecutionPlan, such as 'DataSourceExec'. /// /// Implementation note: this method can just proxy to @@ -546,6 +558,23 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { child_pushdown_result, )) } + + /// Whether this operator supports cooperative yielding. Default is false. + fn yields_cooperatively(&self) -> bool { + false + } + + /// If `yields_cooperatively() == true`, return an Arc to a new version + /// of this plan node that includes built‐in yielding. Otherwise, return None. + /// + /// We have removed any `Self: Sized` bound so that this method can be + /// invoked on a trait object (`Arc`). The default + /// implementation simply returns `None`. Concrete types (e.g. DataSourceExec) + /// should override this method when they really do have a built‐in yielding variant. + fn with_cooperative_yields(self: Arc) -> Option> { + // Default: no built‐in yielding, so return None. + None + } } /// [`ExecutionPlan`] Invariant Level diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index d89ba600d7a6..051dfd886549 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -6034,7 +6034,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec frequency=64 +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6063,7 +6064,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec frequency=64 +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6092,7 +6094,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec frequency=64 +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # FIXME: due to rewrite below not working, this is _extremely_ slow to evaluate # query I @@ -6123,7 +6126,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: array_has([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c], substr(md5(CAST(value@0 AS Utf8)), 1, 32)) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec frequency=64 +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6152,7 +6156,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec frequency=64 +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6183,7 +6188,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IS NOT NULL OR NULL 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------YieldStreamExec frequency=64 +10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # any operator query ? diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 32320a06f4fb..d3267d772553 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -166,7 +166,8 @@ physical_plan 06)--------CoalesceBatchesExec: target_batch_size=2 07)----------FilterExec: time@0 < 10 08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------------WorkTableExec: name=balances +09)--------------YieldStreamExec frequency=64 +10)----------------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 @@ -738,9 +739,11 @@ physical_plan 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 +10)--------------YieldStreamExec frequency=64 +11)----------------WorkTableExec: name=recursive_cte +12)------ProjectionExec: expr=[2 as val] +13)--------YieldStreamExec frequency=64 +14)----------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/9794 # Non-recursive term and recursive term have different types @@ -936,7 +939,8 @@ physical_plan 01)CrossJoinExec 02)--DataSourceExec: partitions=1, partition_sizes=[0] 03)--ProjectionExec: expr=[1 as Int64(1)] -04)----PlaceholderRowExec +04)----YieldStreamExec frequency=64 +05)------PlaceholderRowExec statement count 0 drop table j1; diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 15bf61576571..2e27f0582c09 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -814,13 +814,19 @@ physical_plan 31)│ v1: value │ 32)└─────────────┬─────────────┘ 33)┌─────────────┴─────────────┐ -34)│ LazyMemoryExec │ +34)│ YieldStreamExec │ 35)│ -------------------- │ -36)│ batch_generators: │ -37)│ generate_series: start=1, │ -38)│ end=1000, batch_size │ -39)│ =8192 │ -40)└───────────────────────────┘ +36)│ YieldStreamExec frequency:│ +37)│ 64 │ +38)└─────────────┬─────────────┘ +39)┌─────────────┴─────────────┐ +40)│ LazyMemoryExec │ +41)│ -------------------- │ +42)│ batch_generators: │ +43)│ generate_series: start=1, │ +44)│ end=1000, batch_size │ +45)│ =8192 │ +46)└───────────────────────────┘ query TT explain select diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index b6a32c327f66..4390e7db5fe8 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -284,7 +284,7 @@ datafusion.format.timestamp_tz_format NULL datafusion.format.types_info false datafusion.optimizer.allow_symmetric_joins_without_pruning true datafusion.optimizer.default_filter_selectivity 20 -datafusion.optimizer.enable_add_yield_for_pipeline_break false +datafusion.optimizer.enable_add_yield_for_pipeline_break true datafusion.optimizer.enable_distinct_aggregation_soft_limit true datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.enable_topk_aggregation true @@ -396,7 +396,7 @@ datafusion.format.timestamp_tz_format NULL Timestamp format for timestamp with t datafusion.format.types_info false Show types in visual representation batches datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. datafusion.optimizer.default_filter_selectivity 20 The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). -datafusion.optimizer.enable_add_yield_for_pipeline_break false When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. Default: false (disabled). +datafusion.optimizer.enable_add_yield_for_pipeline_break true When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. Default: true (enabled). datafusion.optimizer.enable_distinct_aggregation_soft_limit true When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index ccecb9494331..86572421d76b 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -4702,7 +4702,8 @@ physical_plan 01)CrossJoinExec 02)--DataSourceExec: partitions=1, partition_sizes=[0] 03)--ProjectionExec: expr=[1 as Int64(1)] -04)----PlaceholderRowExec +04)----YieldStreamExec frequency=64 +05)------PlaceholderRowExec statement count 0 drop table j1; diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index d549f555f9d8..cff5c67b134e 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -520,11 +520,13 @@ physical_plan 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true 16)----ProjectionExec: expr=[1 as cnt] -17)------PlaceholderRowExec -18)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] -19)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) 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 }], mode=[Sorted] -20)--------ProjectionExec: expr=[1 as c1] -21)----------PlaceholderRowExec +17)------YieldStreamExec frequency=64 +18)--------PlaceholderRowExec +19)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] +20)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) 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 }], mode=[Sorted] +21)--------ProjectionExec: expr=[1 as c1] +22)----------YieldStreamExec frequency=64 +23)------------PlaceholderRowExec ######## @@ -659,10 +661,12 @@ physical_plan 02)--ProjectionExec: expr=[count(Int64(1))@1 as count, n@0 as n] 03)----AggregateExec: mode=SinglePartitioned, gby=[n@0 as n], aggr=[count(Int64(1))], ordering_mode=Sorted 04)------ProjectionExec: expr=[5 as n] -05)--------PlaceholderRowExec -06)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] -07)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] -08)------PlaceholderRowExec +05)--------YieldStreamExec frequency=64 +06)----------PlaceholderRowExec +07)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] +08)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] +09)------YieldStreamExec frequency=64 +10)--------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/11409 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index c86921012f9b..8707177633bc 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -280,13 +280,17 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[max(d.a)], ordering_mode=Sorted 08)--------------UnionExec 09)----------------ProjectionExec: expr=[1 as a, aa as b] -10)------------------PlaceholderRowExec -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 +10)------------------YieldStreamExec frequency=64 +11)--------------------PlaceholderRowExec +12)----------------ProjectionExec: expr=[3 as a, aa as b] +13)------------------YieldStreamExec frequency=64 +14)--------------------PlaceholderRowExec +15)----------------ProjectionExec: expr=[5 as a, bb as b] +16)------------------YieldStreamExec frequency=64 +17)--------------------PlaceholderRowExec +18)----------------ProjectionExec: expr=[7 as a, bb as b] +19)------------------YieldStreamExec frequency=64 +20)--------------------PlaceholderRowExec # Check actual result: query TI @@ -366,13 +370,17 @@ physical_plan 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 +11)--------------------YieldStreamExec frequency=64 +12)----------------------PlaceholderRowExec +13)------------------ProjectionExec: expr=[3 as a, aa as b] +14)--------------------YieldStreamExec frequency=64 +15)----------------------PlaceholderRowExec +16)------------------ProjectionExec: expr=[5 as a, bb as b] +17)--------------------YieldStreamExec frequency=64 +18)----------------------PlaceholderRowExec +19)------------------ProjectionExec: expr=[7 as a, bb as b] +20)--------------------YieldStreamExec frequency=64 +21)----------------------PlaceholderRowExec # check actual result diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index b864886c89bf..75b65537db84 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -119,7 +119,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | | datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | | datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | -| datafusion.optimizer.enable_add_yield_for_pipeline_break | false | When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. Default: false (disabled). | +| datafusion.optimizer.enable_add_yield_for_pipeline_break | true | When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. Default: true (enabled). | | datafusion.optimizer.yield_frequency_for_pipeline_break | 64 | Yield frequency in batches, it represents how many batches to process before yielding to the Tokio scheduler. The default value is 64, which means that after processing 64 batches, the execution will yield control back to the Tokio scheduler. This setting is only effective when `enable_add_yield_for_pipeline_break` is set to true. This value should be greater than 0. | | datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | | datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | From 3eb785cac3ce73d7a3d617147070aaff48bf6cb6 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 5 Jun 2025 13:46:48 +0800 Subject: [PATCH 40/74] Add LazyMemoryExec built-in Yield --- .../physical-plan/src/execution_plan.rs | 2 +- datafusion/physical-plan/src/memory.rs | 47 +++++++++++++++++-- datafusion/sqllogictest/test_files/array.slt | 18 +++---- .../sqllogictest/test_files/explain_tree.slt | 18 +++---- 4 files changed, 57 insertions(+), 28 deletions(-) diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index cf60260d08d4..3b09ca48df42 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -569,7 +569,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync + 'static { /// /// We have removed any `Self: Sized` bound so that this method can be /// invoked on a trait object (`Arc`). The default - /// implementation simply returns `None`. Concrete types (e.g. DataSourceExec) + /// implementation simply returns `None`. Concrete types (e.g. DataSourceExec, LazyMemoryExec) /// should override this method when they really do have a built‐in yielding variant. fn with_cooperative_yields(self: Arc) -> Option> { // Default: no built‐in yielding, so return None. diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 1bc872a56e76..5bc53fe099f3 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -35,6 +35,7 @@ use datafusion_execution::memory_pool::MemoryReservation; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; +use crate::yield_stream::YieldStream; use futures::Stream; use parking_lot::RwLock; @@ -139,6 +140,9 @@ pub trait LazyBatchGenerator: Send + Sync + fmt::Debug + fmt::Display { /// /// This plan generates output batches lazily, it doesn't have to buffer all batches /// in memory up front (compared to `MemorySourceConfig`), thus consuming constant memory. +/// We now add a `cooperative` flag to +/// let it optionally yield back to the runtime periodically. +/// Default is `true`, meaning it will yield back to the runtime for cooperative scheduling. pub struct LazyMemoryExec { /// Schema representing the data schema: SchemaRef, @@ -146,6 +150,8 @@ pub struct LazyMemoryExec { batch_generators: Vec>>, /// Plan properties cache storing equivalence properties, partitioning, and execution mode cache: PlanProperties, + /// Indicates whether to enable cooperative yielding mode. + cooperative: bool, } impl LazyMemoryExec { @@ -160,12 +166,21 @@ impl LazyMemoryExec { EmissionType::Incremental, Boundedness::Bounded, ); + Ok(Self { schema, batch_generators: generators, cache, + cooperative: true, // Cooperative yielding mode defaults to true }) } + + /// Set the Yielding mode for the execution plan + /// It defaults to `true`, meaning it will yield back to the runtime for cooperative scheduling. + pub fn with_cooperative_yielding(mut self, cooperative: bool) -> Self { + self.cooperative = cooperative; + self + } } impl fmt::Debug for LazyMemoryExec { @@ -244,7 +259,7 @@ impl ExecutionPlan for LazyMemoryExec { fn execute( &self, partition: usize, - _context: Arc, + context: Arc, ) -> Result { if partition >= self.batch_generators.len() { return internal_err!( @@ -254,10 +269,36 @@ impl ExecutionPlan for LazyMemoryExec { ); } - Ok(Box::pin(LazyMemoryStream { + let stream = Box::pin(LazyMemoryStream { schema: Arc::clone(&self.schema), generator: Arc::clone(&self.batch_generators[partition]), - })) + }); + + // 2. If cooperative == false, return base_stream immediately. + if !self.cooperative { + return Ok(stream); + } + + let frequency = context + .session_config() + .options() + .optimizer + .yield_frequency_for_pipeline_break; + + // 3. If cooperative == true, wrap the stream into a YieldStream. + let yielding_stream = YieldStream::new(stream, frequency); + Ok(Box::pin(yielding_stream)) + } + + /// If `cooperative == true`, return `Some(self.clone())` so the optimizer knows + /// we can replace a plain DataSourceExec with this same node (it already yields). + /// Otherwise, return None. + fn with_cooperative_yields(self: Arc) -> Option> { + if self.cooperative { + Some(self) + } else { + None + } } fn statistics(&self) -> Result { diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 051dfd886549..d89ba600d7a6 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -6034,8 +6034,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec frequency=64 -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6064,8 +6063,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec frequency=64 -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6094,8 +6092,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec frequency=64 -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # FIXME: due to rewrite below not working, this is _extremely_ slow to evaluate # query I @@ -6126,8 +6123,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: array_has([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c], substr(md5(CAST(value@0 AS Utf8)), 1, 32)) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec frequency=64 -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6156,8 +6152,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec frequency=64 -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6188,8 +6183,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IS NOT NULL OR NULL 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------YieldStreamExec frequency=64 -10)------------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # any operator query ? diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 2e27f0582c09..15bf61576571 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -814,19 +814,13 @@ physical_plan 31)│ v1: value │ 32)└─────────────┬─────────────┘ 33)┌─────────────┴─────────────┐ -34)│ YieldStreamExec │ +34)│ LazyMemoryExec │ 35)│ -------------------- │ -36)│ YieldStreamExec frequency:│ -37)│ 64 │ -38)└─────────────┬─────────────┘ -39)┌─────────────┴─────────────┐ -40)│ LazyMemoryExec │ -41)│ -------------------- │ -42)│ batch_generators: │ -43)│ generate_series: start=1, │ -44)│ end=1000, batch_size │ -45)│ =8192 │ -46)└───────────────────────────┘ +36)│ batch_generators: │ +37)│ generate_series: start=1, │ +38)│ end=1000, batch_size │ +39)│ =8192 │ +40)└───────────────────────────┘ query TT explain select From 5737309b43e712c0f3a2408cbfd29e4911d44f01 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 5 Jun 2025 18:34:46 +0800 Subject: [PATCH 41/74] Update datafusion/datasource/src/source.rs Co-authored-by: Mehmet Ozan Kabak --- datafusion/datasource/src/source.rs | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index d973882946dd..ae47c334751f 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -290,11 +290,7 @@ impl ExecutionPlan for DataSourceExec { /// we can replace a plain DataSourceExec with this same node (it already yields). /// Otherwise, return None. fn with_cooperative_yields(self: Arc) -> Option> { - if self.cooperative { - Some(self) - } else { - None - } + self.cooperative.then_some(self) } fn metrics(&self) -> Option { From 74eb5c2e6679c9234958cb87d356f4ffae5d4870 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 5 Jun 2025 18:35:02 +0800 Subject: [PATCH 42/74] Update datafusion/core/tests/physical_optimizer/enforce_distribution.rs Co-authored-by: Mehmet Ozan Kabak --- datafusion/core/tests/physical_optimizer/enforce_distribution.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 3a021c412944..4034800c30cb 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3534,7 +3534,6 @@ async fn test_distribute_sort_memtable() -> Result<()> { let session_config = SessionConfig::new() .with_repartition_file_min_size(1000) .with_target_partitions(3); - let ctx = SessionContext::new_with_config(session_config); ctx.register_table("users", Arc::new(mem_table))?; From 23d7a4b8869828d341e867f87980767947d79b60 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 5 Jun 2025 18:36:06 +0800 Subject: [PATCH 43/74] Update datafusion/physical-optimizer/src/optimizer.rs Co-authored-by: Mehmet Ozan Kabak --- datafusion/physical-optimizer/src/optimizer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index e483abd92474..bab866820318 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -34,8 +34,8 @@ use crate::projection_pushdown::ProjectionPushdown; use crate::sanity_checker::SanityCheckPlan; use crate::topk_aggregation::TopKAggregation; use crate::update_aggr_exprs::OptimizeAggregateOrder; - use crate::wrap_leaves_cancellation::WrapLeaves; + use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_physical_plan::ExecutionPlan; From e7c7eeae18595af43440e2db2e8d759ed5d0c0b4 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 5 Jun 2025 18:36:20 +0800 Subject: [PATCH 44/74] Update datafusion/physical-plan/src/memory.rs Co-authored-by: Mehmet Ozan Kabak --- datafusion/physical-plan/src/memory.rs | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 5bc53fe099f3..9a494dce1be4 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -294,11 +294,7 @@ impl ExecutionPlan for LazyMemoryExec { /// we can replace a plain DataSourceExec with this same node (it already yields). /// Otherwise, return None. fn with_cooperative_yields(self: Arc) -> Option> { - if self.cooperative { - Some(self) - } else { - None - } + self.cooperative.then_some(self) } fn statistics(&self) -> Result { From 7df28e5b47abe885ef4c6e63e20dac6774e0dd52 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 5 Jun 2025 18:36:37 +0800 Subject: [PATCH 45/74] Update datafusion/physical-plan/src/memory.rs Co-authored-by: Mehmet Ozan Kabak --- datafusion/physical-plan/src/memory.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 9a494dce1be4..2eeb8ac6bb3d 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -150,7 +150,7 @@ pub struct LazyMemoryExec { batch_generators: Vec>>, /// Plan properties cache storing equivalence properties, partitioning, and execution mode cache: PlanProperties, - /// Indicates whether to enable cooperative yielding mode. + /// Indicates whether to enable cooperative yielding mode (defaults to `true`). cooperative: bool, } From d16e6e50d8c317be4ed25db0e9704fd2826ebbbf Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 5 Jun 2025 18:36:57 +0800 Subject: [PATCH 46/74] Update datafusion/proto/src/physical_plan/mod.rs Co-authored-by: Mehmet Ozan Kabak --- datafusion/proto/src/physical_plan/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index f701b28d6d3a..74a2c1261eec 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -88,8 +88,8 @@ use datafusion::physical_plan::{ use datafusion_common::config::TableParquetOptions; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; - use datafusion::physical_plan::yield_stream::YieldStreamExec; + use prost::bytes::BufMut; use prost::Message; From f6ccecbe5b7ae435fdc2ad5b547f3676af3ed811 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 5 Jun 2025 21:20:03 +0800 Subject: [PATCH 47/74] Address comments --- datafusion/common/src/config.rs | 7 +++++-- datafusion/core/tests/execution/infinite_cancel.rs | 6 ++---- datafusion/datasource/src/source.rs | 3 --- .../src/wrap_leaves_cancellation.rs | 2 +- datafusion/physical-plan/src/execution_plan.rs | 14 +------------- datafusion/physical-plan/src/memory.rs | 3 --- datafusion/proto/src/physical_plan/mod.rs | 2 +- datafusion/sqllogictest/test_files/group_by.slt | 2 +- .../sqllogictest/test_files/information_schema.slt | 2 +- docs/source/user-guide/configs.md | 2 +- 10 files changed, 13 insertions(+), 30 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index d599e475f5c0..0111d4955eeb 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -732,8 +732,11 @@ config_namespace! { /// Yield frequency in batches, it represents how many batches to process before yielding /// to the Tokio scheduler. The default value is 64, which means that after processing /// 64 batches, the execution will yield control back to the Tokio scheduler. - /// This setting is only effective when `enable_add_yield_for_pipeline_break` is set to true. - /// This value should be greater than 0. + /// This value should be greater than 0 or equal to 0; + /// When it is greater than 0, `enable` the optimizer will insert a Yield operator at the leaf nodes of any pipeline + /// that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to + /// other tasks while waiting. + /// If you want to `disable` the Yield operator, you can set this value to 0. pub yield_frequency_for_pipeline_break: usize, default = 64 } } diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index 8afb6a98d610..3e2e01048336 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -172,8 +172,7 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { )?); // 3) optimize the plan with WrapLeaves to auto-insert Yield - let mut config = ConfigOptions::new(); - config.optimizer.enable_add_yield_for_pipeline_break = true; + let config = ConfigOptions::new(); let optimized = WrapLeaves::new().optimize(aggr.clone(), &config)?; // 4) get the stream @@ -225,8 +224,7 @@ async fn test_infinite_sort_cancel() -> Result<(), Box> { let sort_exec = Arc::new(SortExec::new(lex_ordering, inf.clone())); // 4) optimize the plan with WrapLeaves to auto-insert Yield - let mut config = ConfigOptions::new(); - config.optimizer.enable_add_yield_for_pipeline_break = true; + let config = ConfigOptions::new(); let optimized = WrapLeaves::new().optimize(sort_exec.clone(), &config)?; // 5) get the stream diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index ae47c334751f..1722f3a8af69 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -180,9 +180,6 @@ pub trait DataSource: Send + Sync + Debug { /// the [`FileSource`] trait. /// /// [`FileSource`]: crate::file::FileSource -/// We now add a `cooperative` flag to -/// let it optionally yield back to the runtime periodically. -/// Default is `true`, meaning it will yield back to the runtime for cooperative scheduling. #[derive(Clone, Debug)] pub struct DataSourceExec { /// The source of the data -- for example, `FileScanConfig` or `MemorySourceConfig` diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index 450429eeba1e..06325d6c4ba8 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -116,7 +116,7 @@ impl PhysicalOptimizerRule for WrapLeaves { plan: Arc, config: &ConfigOptions, ) -> Result> { - if config.optimizer.enable_add_yield_for_pipeline_break { + if config.optimizer.yield_frequency_for_pipeline_break != 0 { let yield_frequency = config.optimizer.yield_frequency_for_pipeline_break; // We run a top‐level transform_down: for every node, call wrap_leaves_of_pipeline_breakers. diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 3b09ca48df42..2641b1f47251 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -75,19 +75,7 @@ use futures::stream::{StreamExt, TryStreamExt}; /// [`execute`]: ExecutionPlan::execute /// [`required_input_distribution`]: ExecutionPlan::required_input_distribution /// [`required_input_ordering`]: ExecutionPlan::required_input_ordering -/// The core trait for a physical execution plan node. Every operator -/// implements this trait. We have extended it by adding two new methods -/// for “cooperative yielding” support: -/// -/// 1. `yields_cooperatively()` indicates whether this operator already -/// supports async/yield behavior internally (default: false). -/// -/// 2. `with_cooperative_yields(self: Arc)` returns an alternate -/// plan node that has built-in yielding; if not available, returns None. -/// -/// Because `with_cooperative_yields` moves `Arc` into `Arc`, -/// we must ensure `Self: 'static`. Therefore, we add `+ 'static` here. -pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync + 'static { +pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Short name for the ExecutionPlan, such as 'DataSourceExec'. /// /// Implementation note: this method can just proxy to diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 2eeb8ac6bb3d..cc8404c1e1f9 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -140,9 +140,6 @@ pub trait LazyBatchGenerator: Send + Sync + fmt::Debug + fmt::Display { /// /// This plan generates output batches lazily, it doesn't have to buffer all batches /// in memory up front (compared to `MemorySourceConfig`), thus consuming constant memory. -/// We now add a `cooperative` flag to -/// let it optionally yield back to the runtime periodically. -/// Default is `true`, meaning it will yield back to the runtime for cooperative scheduling. pub struct LazyMemoryExec { /// Schema representing the data schema: SchemaRef, diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 74a2c1261eec..89ac3cc56f6c 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -82,13 +82,13 @@ use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMerge use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::unnest::{ListUnnest, UnnestExec}; use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; +use datafusion::physical_plan::yield_stream::YieldStreamExec; use datafusion::physical_plan::{ ExecutionPlan, InputOrderMode, PhysicalExpr, WindowExpr, }; use datafusion_common::config::TableParquetOptions; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; -use datafusion::physical_plan::yield_stream::YieldStreamExec; use prost::bytes::BufMut; use prost::Message; diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index e6879496427b..9e67018ecd0b 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -4113,7 +4113,7 @@ EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 ---- logical_plan 01)Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 -02)--Cross Join: +02)--Cross Join: 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))]] diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 4390e7db5fe8..9ff628757aa0 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -416,7 +416,7 @@ datafusion.optimizer.repartition_sorts true Should DataFusion execute sorts in a datafusion.optimizer.repartition_windows true Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level datafusion.optimizer.skip_failed_rules false When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail datafusion.optimizer.top_down_join_key_reordering true When set to true, the physical plan optimizer will run a top down process to reorder the join keys -datafusion.optimizer.yield_frequency_for_pipeline_break 64 Yield frequency in batches, it represents how many batches to process before yielding to the Tokio scheduler. The default value is 64, which means that after processing 64 batches, the execution will yield control back to the Tokio scheduler. This setting is only effective when `enable_add_yield_for_pipeline_break` is set to true. This value should be greater than 0. +datafusion.optimizer.yield_frequency_for_pipeline_break 64 Yield frequency in batches, it represents how many batches to process before yielding to the Tokio scheduler. The default value is 64, which means that after processing 64 batches, the execution will yield control back to the Tokio scheduler. This value should be greater than 0 or equal to 0; When it is greater than 0, `enable` the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. If you want to `disable` the Yield operator, you can set this value to 0. datafusion.sql_parser.collect_spans false When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. datafusion.sql_parser.dialect generic Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. datafusion.sql_parser.enable_ident_normalization true When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index e6363ff7ffdb..28587fd8d642 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -120,7 +120,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | | datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | | datafusion.optimizer.enable_add_yield_for_pipeline_break | true | When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. Default: true (enabled). | -| datafusion.optimizer.yield_frequency_for_pipeline_break | 64 | Yield frequency in batches, it represents how many batches to process before yielding to the Tokio scheduler. The default value is 64, which means that after processing 64 batches, the execution will yield control back to the Tokio scheduler. This setting is only effective when `enable_add_yield_for_pipeline_break` is set to true. This value should be greater than 0. | +| datafusion.optimizer.yield_frequency_for_pipeline_break | 64 | Yield frequency in batches, it represents how many batches to process before yielding to the Tokio scheduler. The default value is 64, which means that after processing 64 batches, the execution will yield control back to the Tokio scheduler. This value should be greater than 0 or equal to 0; When it is greater than 0, `enable` the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. If you want to `disable` the Yield operator, you can set this value to 0. | | datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | | datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | | datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | From 8c7462006ec3a7cbd667d02e2b2fb6878812137b Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 5 Jun 2025 22:34:52 +0800 Subject: [PATCH 48/74] Add interleave reproducer --- .../core/tests/execution/infinite_cancel.rs | 118 +++++++++++++++++- 1 file changed, 117 insertions(+), 1 deletion(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index 3e2e01048336..d7e268374744 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -32,11 +32,18 @@ use datafusion::physical_plan::{ use datafusion::prelude::SessionContext; use datafusion::{common, physical_plan}; use datafusion_common::config::ConfigOptions; -use datafusion_physical_expr::expressions::Column; +use datafusion_common::ScalarValue; +use datafusion_expr_common::operator::Operator::Gt; +use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use datafusion_physical_optimizer::wrap_leaves_cancellation::WrapLeaves; use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::union::InterleaveExec; use futures::{Stream, StreamExt}; use std::any::Any; use std::error::Error; @@ -245,3 +252,112 @@ async fn test_infinite_sort_cancel() -> Result<(), Box> { ); Ok(()) } + +#[tokio::test] +async fn test_infinite_interleave_agg_cancel() -> Result<(), Box> { + // 1) Build session, schema, and a sample batch. + let session_ctx = SessionContext::new(); + let schema: SchemaRef = Arc::new(Schema::new(Fields::from(vec![Field::new( + "value", + DataType::Int64, + false, + )]))); + let mut builder = Int64Array::builder(8192); + for v in 0..8192 { + builder.append_value(v); + } + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(builder.finish())])?; + + // 2) Create N infinite sources, each filtered by a different predicate. + // That way, the InterleaveExec will have multiple children. + let mut infinite_children: Vec> = Vec::new(); + let thresholds = [ + 8192, 8111, 8030, 7949, 7868, 7787, 7706, 7625, 7544, 7463, 7382, 7301, 7220, + 7139, 7058, 6977, 6896, 6815, 6734, 6653, 6572, 6491, 6410, 6329, 6248, 6167, + 6086, 6005, 5924, 5843, 5762, 5681, 5600, + ]; + for &thr in &thresholds { + // 2a) One infinite exec: + let inf = Arc::new(InfiniteExec::new(&batch)); + + // 2b) Apply a FilterExec: “value > thr”. + let filter_expr: Arc = Arc::new(BinaryExpr::new( + Arc::new(Column::new_with_schema("value", &schema)?), + Gt, + Arc::new(Literal::new(ScalarValue::Int64(Some(thr)))), + )); + let filtered: Arc = + Arc::new(FilterExec::try_new(filter_expr, inf.clone())?); + + // 2c) Wrap in CoalesceBatchesExec so the upstream yields are batched. + let coalesced: Arc = + Arc::new(CoalesceBatchesExec::new(filtered.clone(), 8192)); + + // 2d) Now repartition so that all children share identical Hash partitioning + // on “value” into 1 bucket. This is required for InterleaveExec::try_new. + let exprs: Vec> = + vec![Arc::new(Column::new_with_schema("value", &schema)?)]; + let partitioning = Partitioning::Hash(exprs.clone(), 1); + let hashed: Arc = Arc::new(RepartitionExec::try_new( + coalesced.clone(), + partitioning.clone(), + )?); + + infinite_children.push(hashed); + } + + // 3) Build an InterleaveExec over all N children. + // Since each child now has Partitioning::Hash([col "value"], 1), InterleaveExec::try_new succeeds. + let interleave: Arc = + Arc::new(InterleaveExec::try_new(infinite_children.clone())?); + + // 4) Build a global AggregateExec that sums “value” over all rows. + // Because we use `AggregateMode::Single` with no GROUP BY columns, this plan will + // only produce one “final” row once all inputs finish. But our inputs never finish, + // so we should never get any output. + let aggregate_expr = AggregateExprBuilder::new( + sum::sum_udaf(), + vec![Arc::new(Column::new_with_schema("value", &schema)?)], + ) + .schema(interleave.schema()) + .alias("total") + .build()?; + + let aggr: Arc = Arc::new(AggregateExec::try_new( + AggregateMode::Single, + PhysicalGroupBy::new( + vec![], // no GROUP BY columns + vec![], // no GROUP BY expressions + vec![], // no GROUP BY physical expressions + ), + vec![Arc::new(aggregate_expr)], + vec![None], // no “distinct” flags + interleave.clone(), + interleave.schema(), + )?); + + // 5) WrapLeaves will automatically insert YieldStreams beneath each “infinite” leaf. + // That way, each InfiniteExec (through the FilterExec/CoalesceBatchesExec/RepartitionExec chain) + // yields to the runtime periodically instead of spinning CPU. + let config = ConfigOptions::new(); + let optimized = WrapLeaves::new().optimize(aggr, &config)?; + + // 6) Execute the stream. Because AggregateExec(mode=Single) only emits a final batch + // after all inputs finish—and those inputs are infinite—we expect no output + // within 1 second (timeout → None). + let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; + const TIMEOUT: u64 = 1; + let result = select! { + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { + None + } + }; + + assert!( + result.is_none(), + "Expected no output for aggregate over infinite interleave, but got some batch" + ); + + Ok(()) +} From 5e2dde841f4d3cf1aaa91f3e58d93d3607121300 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 5 Jun 2025 22:47:29 +0800 Subject: [PATCH 49/74] remove unused config --- datafusion/common/src/config.rs | 6 ------ datafusion/sqllogictest/test_files/information_schema.slt | 2 -- docs/source/user-guide/configs.md | 1 - 3 files changed, 9 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 0111d4955eeb..5645b89449e8 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -723,12 +723,6 @@ config_namespace! { /// Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. pub expand_views_at_output: bool, default = false - /// When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline - /// that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to - /// other tasks while waiting. - /// Default: true (enabled). - pub enable_add_yield_for_pipeline_break: bool, default = true - /// Yield frequency in batches, it represents how many batches to process before yielding /// to the Tokio scheduler. The default value is 64, which means that after processing /// 64 batches, the execution will yield control back to the Tokio scheduler. diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 9ff628757aa0..a484eca6717c 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -284,7 +284,6 @@ datafusion.format.timestamp_tz_format NULL datafusion.format.types_info false datafusion.optimizer.allow_symmetric_joins_without_pruning true datafusion.optimizer.default_filter_selectivity 20 -datafusion.optimizer.enable_add_yield_for_pipeline_break true datafusion.optimizer.enable_distinct_aggregation_soft_limit true datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.enable_topk_aggregation true @@ -396,7 +395,6 @@ datafusion.format.timestamp_tz_format NULL Timestamp format for timestamp with t datafusion.format.types_info false Show types in visual representation batches datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. datafusion.optimizer.default_filter_selectivity 20 The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). -datafusion.optimizer.enable_add_yield_for_pipeline_break true When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. Default: true (enabled). datafusion.optimizer.enable_distinct_aggregation_soft_limit true When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 28587fd8d642..4b05d4827b2d 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -119,7 +119,6 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | | datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | | datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | -| datafusion.optimizer.enable_add_yield_for_pipeline_break | true | When true, the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. Default: true (enabled). | | datafusion.optimizer.yield_frequency_for_pipeline_break | 64 | Yield frequency in batches, it represents how many batches to process before yielding to the Tokio scheduler. The default value is 64, which means that after processing 64 batches, the execution will yield control back to the Tokio scheduler. This value should be greater than 0 or equal to 0; When it is greater than 0, `enable` the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. If you want to `disable` the Yield operator, you can set this value to 0. | | datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | | datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | From aa73a2d005a7534e908e54b2b0a87d0fbd90764e Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 11:23:39 +0800 Subject: [PATCH 50/74] Add join with aggr case --- .../core/tests/execution/infinite_cancel.rs | 138 +++++++++++++++++- 1 file changed, 132 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index d7e268374744..ddf4a97580f5 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -32,7 +32,7 @@ use datafusion::physical_plan::{ use datafusion::prelude::SessionContext; use datafusion::{common, physical_plan}; use datafusion_common::config::ConfigOptions; -use datafusion_common::ScalarValue; +use datafusion_common::{JoinType, ScalarValue}; use datafusion_expr_common::operator::Operator::Gt; use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; @@ -41,6 +41,8 @@ use datafusion_physical_optimizer::wrap_leaves_cancellation::WrapLeaves; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; +use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::union::InterleaveExec; @@ -271,11 +273,11 @@ async fn test_infinite_interleave_agg_cancel() -> Result<(), Box> { // 2) Create N infinite sources, each filtered by a different predicate. // That way, the InterleaveExec will have multiple children. let mut infinite_children: Vec> = Vec::new(); - let thresholds = [ - 8192, 8111, 8030, 7949, 7868, 7787, 7706, 7625, 7544, 7463, 7382, 7301, 7220, - 7139, 7058, 6977, 6896, 6815, 6734, 6653, 6572, 6491, 6410, 6329, 6248, 6167, - 6086, 6005, 5924, 5843, 5762, 5681, 5600, - ]; + // Use 32 distinct thresholds (each >0 and <8 192) to force 32 infinite inputs + let thresholds = (0..32) + .map(|i| 8_192 - 1 - (i * 256) as i64) + .collect::>(); + for &thr in &thresholds { // 2a) One infinite exec: let inf = Arc::new(InfiniteExec::new(&batch)); @@ -361,3 +363,127 @@ async fn test_infinite_interleave_agg_cancel() -> Result<(), Box> { Ok(()) } + +#[tokio::test] +async fn test_infinite_join_agg_cancel() -> Result<(), Box> { + // 1) Session, schema, and a single 8 K‐row batch for each side + let session_ctx = SessionContext::new(); + let schema: SchemaRef = Arc::new(Schema::new(vec![Field::new( + "value", + DataType::Int64, + false, + )])); + + let mut builder_left = Int64Array::builder(8_192); + let mut builder_right = Int64Array::builder(8_192); + for v in 0..8_192 { + builder_left.append_value(v); + // on the right side, we’ll shift each value by +1 so that not everything joins, + // but plenty of matching keys exist (e.g. 0 on left matches 1 on right, etc.) + builder_right.append_value(v + 1); + } + let batch_left = Arc::new(RecordBatch::try_new( + schema.clone(), + vec![Arc::new(builder_left.finish())], + )?); + let batch_right = Arc::new(RecordBatch::try_new( + schema.clone(), + vec![Arc::new(builder_right.finish())], + )?); + + // 2a) Build two InfiniteExecs (left and right) + let infinite_left = Arc::new(InfiniteExec::new(&*batch_left)); + let infinite_right = Arc::new(InfiniteExec::new(&*batch_right)); + + // 2b) Create Join keys → join on “value” = “value” + let left_keys: Vec> = + vec![Arc::new(Column::new_with_schema("value", &schema)?)]; + let right_keys: Vec> = + vec![Arc::new(Column::new_with_schema("value", &schema)?)]; + + // 2c) Wrap each side in CoalesceBatches + Repartition so they are both hashed into 1 partition + let coalesced_left: Arc = + Arc::new(CoalesceBatchesExec::new(infinite_left.clone(), 8_192)); + let coalesced_right: Arc = + Arc::new(CoalesceBatchesExec::new(infinite_right.clone(), 8_192)); + + let part_left = Partitioning::Hash(left_keys.clone(), 1); + let part_right = Partitioning::Hash(right_keys.clone(), 1); + + let hashed_left: Arc = + Arc::new(RepartitionExec::try_new(coalesced_left.clone(), part_left)?); + let hashed_right: Arc = Arc::new(RepartitionExec::try_new( + coalesced_right.clone(), + part_right, + )?); + + // 2d) Build an Inner HashJoinExec → left.value = right.value + let join: Arc = Arc::new(HashJoinExec::try_new( + hashed_left.clone(), + hashed_right.clone(), + vec![( + Arc::new(Column::new_with_schema("value", &hashed_left.schema()).unwrap()), + Arc::new(Column::new_with_schema("value", &hashed_right.schema()).unwrap()), + )], + None, + &JoinType::Inner, + None, + PartitionMode::CollectLeft, + true, + )?); + + // 3) Project only one column (“value” from the left side) because we just want to sum that + let input_schema = join.schema(); + + let proj_expr: Vec<(Arc, String)> = vec![( + Arc::new(Column::new_with_schema("value", &input_schema)?) + as Arc, + "value".to_string(), + )]; + + let projection: Arc = + Arc::new(ProjectionExec::try_new(proj_expr, join.clone())?); + + let output_fields = vec![Field::new("total", DataType::Int64, true)]; + let output_schema = Arc::new(Schema::new(output_fields)); + + // 4) Global aggregate (Single) over “value” + let aggregate_expr = AggregateExprBuilder::new( + sum::sum_udaf(), + vec![Arc::new(Column::new_with_schema( + "value", + &projection.schema(), + )?)], + ) + .schema(output_schema.clone()) + .alias("total") + .build()?; + + let aggr: Arc = Arc::new(AggregateExec::try_new( + AggregateMode::Single, + PhysicalGroupBy::new(vec![], vec![], vec![]), + vec![Arc::new(aggregate_expr)], + vec![None], + projection.clone(), + projection.schema(), + )?); + + // 5) Wrap yields under each infinite leaf + let config = ConfigOptions::new(); + let optimized = WrapLeaves::new().optimize(aggr, &config)?; + + // 6) Execute + 1 sec timeout + let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; + const TIMEOUT: u64 = 1; + let result = select! { + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { + None + } + }; + assert!( + result.is_none(), + "Expected no output for aggregate over infinite + join, but got a batch" + ); + Ok(()) +} From b825ea6ffd6fb3dbc7770d85956bb6f58e22e561 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 11:38:45 +0800 Subject: [PATCH 51/74] fix clippy --- datafusion/core/tests/execution/infinite_cancel.rs | 4 ++-- datafusion/physical-plan/src/memory.rs | 4 +--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index ddf4a97580f5..7943cd9143f4 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -392,8 +392,8 @@ async fn test_infinite_join_agg_cancel() -> Result<(), Box> { )?); // 2a) Build two InfiniteExecs (left and right) - let infinite_left = Arc::new(InfiniteExec::new(&*batch_left)); - let infinite_right = Arc::new(InfiniteExec::new(&*batch_right)); + let infinite_left = Arc::new(InfiniteExec::new(&batch_left)); + let infinite_right = Arc::new(InfiniteExec::new(&batch_right)); // 2b) Create Join keys → join on “value” = “value” let left_keys: Vec> = diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 3f638e568f49..60117042c3e7 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -270,9 +270,8 @@ impl ExecutionPlan for LazyMemoryExec { ); } - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); - + let stream = Box::pin(LazyMemoryStream { schema: Arc::clone(&self.schema), generator: Arc::clone(&self.batch_generators[partition]), @@ -300,7 +299,6 @@ impl ExecutionPlan for LazyMemoryExec { /// Otherwise, return None. fn with_cooperative_yields(self: Arc) -> Option> { self.cooperative.then_some(self) - } fn metrics(&self) -> Option { From be88233cddc954690aa749ef7e7b4dc2cad1b7c1 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 12:18:28 +0800 Subject: [PATCH 52/74] add reproducer for filter and add workaround in rule --- .../core/tests/execution/infinite_cancel.rs | 65 +++++++++++++++++++ .../src/wrap_leaves_cancellation.rs | 4 +- 2 files changed, 68 insertions(+), 1 deletion(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index 7943cd9143f4..c378293dd802 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -487,3 +487,68 @@ async fn test_infinite_join_agg_cancel() -> Result<(), Box> { ); Ok(()) } + +#[tokio::test] +async fn test_filter_reject_all_batches_cancel() -> Result<(), Box> { + // 1) Create a Session, Schema, and an 8K-row RecordBatch + let session_ctx = SessionContext::new(); + let schema: SchemaRef = Arc::new(Schema::new(vec![Field::new( + "value", + DataType::Int64, + false, + )])); + + // Build a batch with values 0..8191 + let mut builder = Int64Array::builder(8_192); + for v in 0..8_192 { + builder.append_value(v); + } + let batch = Arc::new(RecordBatch::try_new( + schema.clone(), + vec![Arc::new(builder.finish())], + )?); + + // 2a) Wrap this batch in an InfiniteExec + let infinite = Arc::new(InfiniteExec::new(&batch)); + + // 2b) Construct a FilterExec that is always false: “value > 10000” (no rows pass) + let false_predicate: Arc = Arc::new(BinaryExpr::new( + Arc::new(Column::new_with_schema("value", &schema)?), + Gt, + Arc::new(Literal::new(ScalarValue::Int64(Some(10_000)))), + )); + let filtered: Arc = + Arc::new(FilterExec::try_new(false_predicate, infinite.clone())?); + + // 2c) Use CoalesceBatchesExec to guarantee each Filter pull always yields an 8192-row batch + let coalesced: Arc = + Arc::new(CoalesceBatchesExec::new(filtered.clone(), 8_192)); + + // 2d) Hash-repartition into 1 partition (so that a later global aggregation would run on a single partition) + let exprs: Vec> = + vec![Arc::new(Column::new_with_schema("value", &schema)?)]; + let part = Partitioning::Hash(exprs.clone(), 1); + let hashed: Arc = + Arc::new(RepartitionExec::try_new(coalesced.clone(), part.clone())?); + + // 4) WrapLeaves to insert YieldExec—so that the InfiniteExec yields control between batches + let config = ConfigOptions::new(); + let optimized = WrapLeaves::new().optimize(hashed, &config)?; + + // 5) Execute with a 1-second timeout. Because Filter discards all 8192 rows each time + // without ever producing output, no batch will arrive within 1 second. And since + // emission type is not Final, we never see an end‐of‐stream marker. + let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; + const TIMEOUT: u64 = 1; + let result = select! { + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { + None + } + }; + assert!( + result.is_none(), + "Expected no output for infinite + filter(all-false) + aggregate, but got a batch" + ); + Ok(()) +} diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index 06325d6c4ba8..f0ea5ff8f3f4 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -24,6 +24,7 @@ use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::ExecutionPlan; use std::fmt::{Debug, Formatter}; use std::sync::Arc; +use datafusion_physical_plan::filter::FilterExec; /// WrapLeaves is a PhysicalOptimizerRule that finds every /// pipeline‐breaking node (emission_type == Final) and then @@ -76,7 +77,8 @@ impl WrapLeaves { plan: Arc, yield_frequency: usize, ) -> Result>> { - let is_pipeline_breaker = plan.properties().emission_type == EmissionType::Final; + // todo this is a bit of a hack, we should probably have a more explicit way to handle + let is_pipeline_breaker = plan.properties().emission_type == EmissionType::Final|| plan.as_any().is::(); if is_pipeline_breaker { // Transform all leaf descendants of this node by calling wrap_leaves let mut transformed = From 4018f8f3d993924f64169908d8ecb14ba3e4cf9f Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 15:56:25 +0800 Subject: [PATCH 53/74] Add reproducer and comment it --- .../core/tests/execution/infinite_cancel.rs | 128 +++++++++--------- .../src/wrap_leaves_cancellation.rs | 4 +- 2 files changed, 65 insertions(+), 67 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index c378293dd802..e1e8d87fd8cb 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -488,67 +488,67 @@ async fn test_infinite_join_agg_cancel() -> Result<(), Box> { Ok(()) } -#[tokio::test] -async fn test_filter_reject_all_batches_cancel() -> Result<(), Box> { - // 1) Create a Session, Schema, and an 8K-row RecordBatch - let session_ctx = SessionContext::new(); - let schema: SchemaRef = Arc::new(Schema::new(vec![Field::new( - "value", - DataType::Int64, - false, - )])); - - // Build a batch with values 0..8191 - let mut builder = Int64Array::builder(8_192); - for v in 0..8_192 { - builder.append_value(v); - } - let batch = Arc::new(RecordBatch::try_new( - schema.clone(), - vec![Arc::new(builder.finish())], - )?); - - // 2a) Wrap this batch in an InfiniteExec - let infinite = Arc::new(InfiniteExec::new(&batch)); - - // 2b) Construct a FilterExec that is always false: “value > 10000” (no rows pass) - let false_predicate: Arc = Arc::new(BinaryExpr::new( - Arc::new(Column::new_with_schema("value", &schema)?), - Gt, - Arc::new(Literal::new(ScalarValue::Int64(Some(10_000)))), - )); - let filtered: Arc = - Arc::new(FilterExec::try_new(false_predicate, infinite.clone())?); - - // 2c) Use CoalesceBatchesExec to guarantee each Filter pull always yields an 8192-row batch - let coalesced: Arc = - Arc::new(CoalesceBatchesExec::new(filtered.clone(), 8_192)); - - // 2d) Hash-repartition into 1 partition (so that a later global aggregation would run on a single partition) - let exprs: Vec> = - vec![Arc::new(Column::new_with_schema("value", &schema)?)]; - let part = Partitioning::Hash(exprs.clone(), 1); - let hashed: Arc = - Arc::new(RepartitionExec::try_new(coalesced.clone(), part.clone())?); - - // 4) WrapLeaves to insert YieldExec—so that the InfiniteExec yields control between batches - let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(hashed, &config)?; - - // 5) Execute with a 1-second timeout. Because Filter discards all 8192 rows each time - // without ever producing output, no batch will arrive within 1 second. And since - // emission type is not Final, we never see an end‐of‐stream marker. - let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; - const TIMEOUT: u64 = 1; - let result = select! { - batch_opt = stream.next() => batch_opt, - _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { - None - } - }; - assert!( - result.is_none(), - "Expected no output for infinite + filter(all-false) + aggregate, but got a batch" - ); - Ok(()) -} +// #[tokio::test] +// async fn test_filter_reject_all_batches_cancel() -> Result<(), Box> { +// // 1) Create a Session, Schema, and an 8K-row RecordBatch +// let session_ctx = SessionContext::new(); +// let schema: SchemaRef = Arc::new(Schema::new(vec![Field::new( +// "value", +// DataType::Int64, +// false, +// )])); +// +// // Build a batch with values 0..8191 +// let mut builder = Int64Array::builder(8_192); +// for v in 0..8_192 { +// builder.append_value(v); +// } +// let batch = Arc::new(RecordBatch::try_new( +// schema.clone(), +// vec![Arc::new(builder.finish())], +// )?); +// +// // 2a) Wrap this batch in an InfiniteExec +// let infinite = Arc::new(InfiniteExec::new(&batch)); +// +// // 2b) Construct a FilterExec that is always false: “value > 10000” (no rows pass) +// let false_predicate: Arc = Arc::new(BinaryExpr::new( +// Arc::new(Column::new_with_schema("value", &schema)?), +// Gt, +// Arc::new(Literal::new(ScalarValue::Int64(Some(10_000)))), +// )); +// let filtered: Arc = +// Arc::new(FilterExec::try_new(false_predicate, infinite.clone())?); +// +// // 2c) Use CoalesceBatchesExec to guarantee each Filter pull always yields an 8192-row batch +// let coalesced: Arc = +// Arc::new(CoalesceBatchesExec::new(filtered.clone(), 8_192)); +// +// // 2d) Hash-repartition into 1 partition (so that a later global aggregation would run on a single partition) +// let exprs: Vec> = +// vec![Arc::new(Column::new_with_schema("value", &schema)?)]; +// let part = Partitioning::Hash(exprs.clone(), 1); +// let hashed: Arc = +// Arc::new(RepartitionExec::try_new(coalesced.clone(), part.clone())?); +// +// // 4) WrapLeaves to insert YieldExec—so that the InfiniteExec yields control between batches +// let config = ConfigOptions::new(); +// let optimized = WrapLeaves::new().optimize(hashed, &config)?; +// +// // 5) Execute with a 1-second timeout. Because Filter discards all 8192 rows each time +// // without ever producing output, no batch will arrive within 1 second. And since +// // emission type is not Final, we never see an end‐of‐stream marker. +// let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; +// const TIMEOUT: u64 = 1; +// let result = select! { +// batch_opt = stream.next() => batch_opt, +// _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { +// None +// } +// }; +// assert!( +// result.is_none(), +// "Expected no output for infinite + filter(all-false) + aggregate, but got a batch" +// ); +// Ok(()) +// } diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index f0ea5ff8f3f4..06325d6c4ba8 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -24,7 +24,6 @@ use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::ExecutionPlan; use std::fmt::{Debug, Formatter}; use std::sync::Arc; -use datafusion_physical_plan::filter::FilterExec; /// WrapLeaves is a PhysicalOptimizerRule that finds every /// pipeline‐breaking node (emission_type == Final) and then @@ -77,8 +76,7 @@ impl WrapLeaves { plan: Arc, yield_frequency: usize, ) -> Result>> { - // todo this is a bit of a hack, we should probably have a more explicit way to handle - let is_pipeline_breaker = plan.properties().emission_type == EmissionType::Final|| plan.as_any().is::(); + let is_pipeline_breaker = plan.properties().emission_type == EmissionType::Final; if is_pipeline_breaker { // Transform all leaf descendants of this node by calling wrap_leaves let mut transformed = From 3b70846a4d9437bfe648317c4aacc8691c7270c6 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 16:09:51 +0800 Subject: [PATCH 54/74] adjust test --- datafusion/core/tests/execution/infinite_cancel.rs | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index e1e8d87fd8cb..824889502883 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -524,18 +524,11 @@ async fn test_infinite_join_agg_cancel() -> Result<(), Box> { // let coalesced: Arc = // Arc::new(CoalesceBatchesExec::new(filtered.clone(), 8_192)); // -// // 2d) Hash-repartition into 1 partition (so that a later global aggregation would run on a single partition) -// let exprs: Vec> = -// vec![Arc::new(Column::new_with_schema("value", &schema)?)]; -// let part = Partitioning::Hash(exprs.clone(), 1); -// let hashed: Arc = -// Arc::new(RepartitionExec::try_new(coalesced.clone(), part.clone())?); -// -// // 4) WrapLeaves to insert YieldExec—so that the InfiniteExec yields control between batches +// // 3) WrapLeaves to insert YieldExec—so that the InfiniteExec yields control between batches // let config = ConfigOptions::new(); -// let optimized = WrapLeaves::new().optimize(hashed, &config)?; +// let optimized = WrapLeaves::new().optimize(coalesced, &config)?; // -// // 5) Execute with a 1-second timeout. Because Filter discards all 8192 rows each time +// // 4) Execute with a 1-second timeout. Because Filter discards all 8192 rows each time // // without ever producing output, no batch will arrive within 1 second. And since // // emission type is not Final, we never see an end‐of‐stream marker. // let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; From 11035e53a7d8143b94af67690c25d68cd3e4a9e2 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 6 Jun 2025 15:24:09 +0300 Subject: [PATCH 55/74] Harden tests, add failing tests for TDD, minor code refactors --- .../core/tests/execution/infinite_cancel.rs | 448 ++++++++++++------ datafusion/physical-plan/src/yield_stream.rs | 97 ++-- datafusion/proto/src/physical_plan/mod.rs | 2 +- 3 files changed, 350 insertions(+), 197 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index 824889502883..83fdde278555 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -15,6 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::error::Error; +use std::fmt::Formatter; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + use arrow::array::{Int64Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use arrow_schema::SortOptions; @@ -46,13 +53,9 @@ use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::union::InterleaveExec; + use futures::{Stream, StreamExt}; -use std::any::Any; -use std::error::Error; -use std::fmt::Formatter; -use std::pin::Pin; -use std::sync::Arc; -use std::task::{Context, Poll}; +use rstest::rstest; use tokio::select; struct InfiniteStream { @@ -85,18 +88,20 @@ struct InfiniteExec { } impl InfiniteExec { - fn new(batch: &RecordBatch) -> Self { - InfiniteExec { - batch: batch.clone(), - properties: PlanProperties::new( - EquivalenceProperties::new(batch.schema().clone()), - Partitioning::UnknownPartitioning(1), - EmissionType::Incremental, + fn new(batch: RecordBatch, pretend_finite: bool) -> Self { + let properties = PlanProperties::new( + EquivalenceProperties::new(batch.schema().clone()), + Partitioning::UnknownPartitioning(1), + EmissionType::Incremental, + if pretend_finite { + Boundedness::Bounded + } else { Boundedness::Unbounded { requires_infinite_memory: false, - }, - ), - } + } + }, + ); + InfiniteExec { batch, properties } } } @@ -146,8 +151,11 @@ impl ExecutionPlan for InfiniteExec { } } +#[rstest] #[tokio::test] -async fn test_infinite_agg_cancel() -> Result<(), Box> { +async fn test_infinite_agg_cancel( + #[values(false, true)] pretend_finite: bool, +) -> Result<(), Box> { // 1) build session & schema & sample batch let session_ctx = SessionContext::new(); let schema = Arc::new(Schema::new(Fields::from(vec![Field::new( @@ -162,7 +170,7 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(builder.finish())])?; // 2) set up the infinite source + aggregation - let inf = Arc::new(InfiniteExec::new(&batch)); + let inf = Arc::new(InfiniteExec::new(batch, pretend_finite)); let aggr = Arc::new(AggregateExec::try_new( AggregateMode::Single, PhysicalGroupBy::new(vec![], vec![], vec![]), @@ -176,13 +184,13 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { .build()?, )], vec![None], - inf.clone(), - inf.schema(), + inf, + schema, )?); // 3) optimize the plan with WrapLeaves to auto-insert Yield let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(aggr.clone(), &config)?; + let optimized = WrapLeaves::new().optimize(aggr, &config)?; // 4) get the stream let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; @@ -200,11 +208,14 @@ async fn test_infinite_agg_cancel() -> Result<(), Box> { Ok(()) } +#[rstest] #[tokio::test] -async fn test_infinite_sort_cancel() -> Result<(), Box> { +async fn test_infinite_sort_cancel( + #[values(false, true)] pretend_finite: bool, +) -> Result<(), Box> { // 1) build session & schema & sample batch let session_ctx = SessionContext::new(); - let schema: SchemaRef = Arc::new(Schema::new(vec![Field::new( + let schema = Arc::new(Schema::new(vec![Field::new( "value", DataType::Int64, false, @@ -217,7 +228,7 @@ async fn test_infinite_sort_cancel() -> Result<(), Box> { let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(array)])?; // 2) set up the infinite source - let inf = Arc::new(InfiniteExec::new(&batch)); + let inf = Arc::new(InfiniteExec::new(batch, pretend_finite)); // 3) set up a SortExec that will never finish because input is infinite let sort_options = SortOptions { @@ -228,13 +239,12 @@ async fn test_infinite_sort_cancel() -> Result<(), Box> { Arc::new(Column::new_with_schema("value", &schema)?), sort_options, ); - // LexOrdering is just Vec let lex_ordering: datafusion::physical_expr::LexOrdering = vec![sort_expr].into(); - let sort_exec = Arc::new(SortExec::new(lex_ordering, inf.clone())); + let sort_exec = Arc::new(SortExec::new(lex_ordering, inf)); // 4) optimize the plan with WrapLeaves to auto-insert Yield let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(sort_exec.clone(), &config)?; + let optimized = WrapLeaves::new().optimize(sort_exec, &config)?; // 5) get the stream let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; @@ -255,11 +265,93 @@ async fn test_infinite_sort_cancel() -> Result<(), Box> { Ok(()) } +#[rstest] +#[tokio::test] +#[ignore] +async fn test_infinite_interleave_cancel( + #[values(false, true)] pretend_finite: bool, +) -> Result<(), Box> { + // 1) Build session, schema, and a sample batch. + let session_ctx = SessionContext::new(); + let schema = Arc::new(Schema::new(Fields::from(vec![Field::new( + "value", + DataType::Int64, + false, + )]))); + let mut builder = Int64Array::builder(8192); + for v in 0..8192 { + builder.append_value(v); + } + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(builder.finish())])?; + + // 2) Create N infinite sources, each filtered by a different predicate. + // That way, the InterleaveExec will have multiple children. + let mut infinite_children = vec![]; + // Use 32 distinct thresholds (each >0 and <8 192) to force 32 infinite inputs + let thresholds = (0..32).map(|i| 8_192 - 1 - (i * 256) as i64); + + for thr in thresholds { + // 2a) One infinite exec: + let inf = Arc::new(InfiniteExec::new(batch.clone(), pretend_finite)); + + // 2b) Apply a FilterExec: “value > thr”. + let filter_expr = Arc::new(BinaryExpr::new( + Arc::new(Column::new_with_schema("value", &schema)?), + Gt, + Arc::new(Literal::new(ScalarValue::Int64(Some(thr)))), + )); + let filtered = Arc::new(FilterExec::try_new(filter_expr, inf)?); + + // 2c) Wrap in CoalesceBatchesExec so the upstream yields are batched. + let coalesced = Arc::new(CoalesceBatchesExec::new(filtered, 8192)); + + // 2d) Now repartition so that all children share identical Hash partitioning + // on “value” into 1 bucket. This is required for InterleaveExec::try_new. + let exprs = vec![Arc::new(Column::new_with_schema("value", &schema)?) as _]; + let partitioning = Partitioning::Hash(exprs, 1); + let hashed = Arc::new(RepartitionExec::try_new(coalesced, partitioning)?); + + infinite_children.push(hashed as _); + } + + // 3) Build an InterleaveExec over all N children. + // Since each child now has Partitioning::Hash([col "value"], 1), InterleaveExec::try_new succeeds. + let interleave = Arc::new(InterleaveExec::try_new(infinite_children)?); + + // 5) WrapLeaves will automatically insert YieldStreams beneath each “infinite” leaf. + // That way, each InfiniteExec (through the FilterExec/CoalesceBatchesExec/RepartitionExec chain) + // yields to the runtime periodically instead of spinning CPU. + let config = ConfigOptions::new(); + let optimized = WrapLeaves::new().optimize(interleave, &config)?; + + // 6) Execute the stream. Because AggregateExec(mode=Single) only emits a final batch + // after all inputs finish—and those inputs are infinite—we expect no output + // within 1 second (timeout → None). + let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; + const TIMEOUT: u64 = 1; + let result = select! { + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { + None + } + }; + + assert!( + result.is_none(), + "Expected no output for aggregate over infinite interleave, but got some batch" + ); + + Ok(()) +} + +#[rstest] #[tokio::test] -async fn test_infinite_interleave_agg_cancel() -> Result<(), Box> { +async fn test_infinite_interleave_agg_cancel( + #[values(false, true)] pretend_finite: bool, +) -> Result<(), Box> { // 1) Build session, schema, and a sample batch. let session_ctx = SessionContext::new(); - let schema: SchemaRef = Arc::new(Schema::new(Fields::from(vec![Field::new( + let schema = Arc::new(Schema::new(Fields::from(vec![Field::new( "value", DataType::Int64, false, @@ -272,46 +364,38 @@ async fn test_infinite_interleave_agg_cancel() -> Result<(), Box> { // 2) Create N infinite sources, each filtered by a different predicate. // That way, the InterleaveExec will have multiple children. - let mut infinite_children: Vec> = Vec::new(); + let mut infinite_children = vec![]; // Use 32 distinct thresholds (each >0 and <8 192) to force 32 infinite inputs - let thresholds = (0..32) - .map(|i| 8_192 - 1 - (i * 256) as i64) - .collect::>(); + let thresholds = (0..32).map(|i| 8_192 - 1 - (i * 256) as i64); - for &thr in &thresholds { + for thr in thresholds { // 2a) One infinite exec: - let inf = Arc::new(InfiniteExec::new(&batch)); + let inf = Arc::new(InfiniteExec::new(batch.clone(), pretend_finite)); // 2b) Apply a FilterExec: “value > thr”. - let filter_expr: Arc = Arc::new(BinaryExpr::new( + let filter_expr = Arc::new(BinaryExpr::new( Arc::new(Column::new_with_schema("value", &schema)?), Gt, Arc::new(Literal::new(ScalarValue::Int64(Some(thr)))), )); - let filtered: Arc = - Arc::new(FilterExec::try_new(filter_expr, inf.clone())?); + let filtered = Arc::new(FilterExec::try_new(filter_expr, inf)?); // 2c) Wrap in CoalesceBatchesExec so the upstream yields are batched. - let coalesced: Arc = - Arc::new(CoalesceBatchesExec::new(filtered.clone(), 8192)); + let coalesced = Arc::new(CoalesceBatchesExec::new(filtered, 8192)); // 2d) Now repartition so that all children share identical Hash partitioning // on “value” into 1 bucket. This is required for InterleaveExec::try_new. - let exprs: Vec> = - vec![Arc::new(Column::new_with_schema("value", &schema)?)]; - let partitioning = Partitioning::Hash(exprs.clone(), 1); - let hashed: Arc = Arc::new(RepartitionExec::try_new( - coalesced.clone(), - partitioning.clone(), - )?); - - infinite_children.push(hashed); + let exprs = vec![Arc::new(Column::new_with_schema("value", &schema)?) as _]; + let partitioning = Partitioning::Hash(exprs, 1); + let hashed = Arc::new(RepartitionExec::try_new(coalesced, partitioning)?); + + infinite_children.push(hashed as _); } // 3) Build an InterleaveExec over all N children. // Since each child now has Partitioning::Hash([col "value"], 1), InterleaveExec::try_new succeeds. - let interleave: Arc = - Arc::new(InterleaveExec::try_new(infinite_children.clone())?); + let interleave = Arc::new(InterleaveExec::try_new(infinite_children)?); + let interleave_schema = interleave.schema(); // 4) Build a global AggregateExec that sums “value” over all rows. // Because we use `AggregateMode::Single` with no GROUP BY columns, this plan will @@ -321,11 +405,11 @@ async fn test_infinite_interleave_agg_cancel() -> Result<(), Box> { sum::sum_udaf(), vec![Arc::new(Column::new_with_schema("value", &schema)?)], ) - .schema(interleave.schema()) + .schema(interleave_schema.clone()) .alias("total") .build()?; - let aggr: Arc = Arc::new(AggregateExec::try_new( + let aggr = Arc::new(AggregateExec::try_new( AggregateMode::Single, PhysicalGroupBy::new( vec![], // no GROUP BY columns @@ -334,8 +418,8 @@ async fn test_infinite_interleave_agg_cancel() -> Result<(), Box> { ), vec![Arc::new(aggregate_expr)], vec![None], // no “distinct” flags - interleave.clone(), - interleave.schema(), + interleave, + interleave_schema, )?); // 5) WrapLeaves will automatically insert YieldStreams beneath each “infinite” leaf. @@ -364,8 +448,11 @@ async fn test_infinite_interleave_agg_cancel() -> Result<(), Box> { Ok(()) } +#[rstest] #[tokio::test] -async fn test_infinite_join_agg_cancel() -> Result<(), Box> { +async fn test_infinite_join_cancel( + #[values(false, true)] pretend_finite: bool, +) -> Result<(), Box> { // 1) Session, schema, and a single 8 K‐row batch for each side let session_ctx = SessionContext::new(); let schema: SchemaRef = Arc::new(Schema::new(vec![Field::new( @@ -382,18 +469,95 @@ async fn test_infinite_join_agg_cancel() -> Result<(), Box> { // but plenty of matching keys exist (e.g. 0 on left matches 1 on right, etc.) builder_right.append_value(v + 1); } - let batch_left = Arc::new(RecordBatch::try_new( - schema.clone(), - vec![Arc::new(builder_left.finish())], - )?); - let batch_right = Arc::new(RecordBatch::try_new( - schema.clone(), - vec![Arc::new(builder_right.finish())], + let batch_left = + RecordBatch::try_new(schema.clone(), vec![Arc::new(builder_left.finish())])?; + let batch_right = + RecordBatch::try_new(schema.clone(), vec![Arc::new(builder_right.finish())])?; + + // 2a) Build two InfiniteExecs (left and right) + let infinite_left = Arc::new(InfiniteExec::new(batch_left, pretend_finite)); + let infinite_right = Arc::new(InfiniteExec::new(batch_right, pretend_finite)); + + // 2b) Create Join keys → join on “value” = “value” + let left_keys: Vec> = + vec![Arc::new(Column::new_with_schema("value", &schema)?)]; + let right_keys: Vec> = + vec![Arc::new(Column::new_with_schema("value", &schema)?)]; + + // 2c) Wrap each side in CoalesceBatches + Repartition so they are both hashed into 1 partition + let coalesced_left = Arc::new(CoalesceBatchesExec::new(infinite_left, 8_192)); + let coalesced_right = Arc::new(CoalesceBatchesExec::new(infinite_right, 8_192)); + + let part_left = Partitioning::Hash(left_keys, 1); + let part_right = Partitioning::Hash(right_keys, 1); + + let hashed_left = Arc::new(RepartitionExec::try_new(coalesced_left, part_left)?); + let hashed_right = Arc::new(RepartitionExec::try_new(coalesced_right, part_right)?); + + // 2d) Build an Inner HashJoinExec → left.value = right.value + let join = Arc::new(HashJoinExec::try_new( + hashed_left, + hashed_right, + vec![( + Arc::new(Column::new_with_schema("value", &schema)?), + Arc::new(Column::new_with_schema("value", &schema)?), + )], + None, + &JoinType::Inner, + None, + PartitionMode::CollectLeft, + true, )?); + // 3) Wrap yields under each infinite leaf + let config = ConfigOptions::new(); + let optimized = WrapLeaves::new().optimize(join, &config)?; + + // 4) Execute + 1 sec timeout + let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; + const TIMEOUT: u64 = 1; + let result = select! { + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { + None + } + }; + assert!( + result.is_none(), + "Expected no output for aggregate over infinite + join, but got a batch" + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_infinite_join_agg_cancel( + #[values(false, true)] pretend_finite: bool, +) -> Result<(), Box> { + // 1) Session, schema, and a single 8 K‐row batch for each side + let session_ctx = SessionContext::new(); + let schema = Arc::new(Schema::new(vec![Field::new( + "value", + DataType::Int64, + false, + )])); + + let mut builder_left = Int64Array::builder(8_192); + let mut builder_right = Int64Array::builder(8_192); + for v in 0..8_192 { + builder_left.append_value(v); + // on the right side, we’ll shift each value by +1 so that not everything joins, + // but plenty of matching keys exist (e.g. 0 on left matches 1 on right, etc.) + builder_right.append_value(v + 1); + } + let batch_left = + RecordBatch::try_new(schema.clone(), vec![Arc::new(builder_left.finish())])?; + let batch_right = + RecordBatch::try_new(schema.clone(), vec![Arc::new(builder_right.finish())])?; + // 2a) Build two InfiniteExecs (left and right) - let infinite_left = Arc::new(InfiniteExec::new(&batch_left)); - let infinite_right = Arc::new(InfiniteExec::new(&batch_right)); + let infinite_left = Arc::new(InfiniteExec::new(batch_left, pretend_finite)); + let infinite_right = Arc::new(InfiniteExec::new(batch_right, pretend_finite)); // 2b) Create Join keys → join on “value” = “value” let left_keys: Vec> = @@ -402,28 +566,22 @@ async fn test_infinite_join_agg_cancel() -> Result<(), Box> { vec![Arc::new(Column::new_with_schema("value", &schema)?)]; // 2c) Wrap each side in CoalesceBatches + Repartition so they are both hashed into 1 partition - let coalesced_left: Arc = - Arc::new(CoalesceBatchesExec::new(infinite_left.clone(), 8_192)); - let coalesced_right: Arc = - Arc::new(CoalesceBatchesExec::new(infinite_right.clone(), 8_192)); + let coalesced_left = Arc::new(CoalesceBatchesExec::new(infinite_left, 8_192)); + let coalesced_right = Arc::new(CoalesceBatchesExec::new(infinite_right, 8_192)); let part_left = Partitioning::Hash(left_keys.clone(), 1); let part_right = Partitioning::Hash(right_keys.clone(), 1); - let hashed_left: Arc = - Arc::new(RepartitionExec::try_new(coalesced_left.clone(), part_left)?); - let hashed_right: Arc = Arc::new(RepartitionExec::try_new( - coalesced_right.clone(), - part_right, - )?); + let hashed_left = Arc::new(RepartitionExec::try_new(coalesced_left, part_left)?); + let hashed_right = Arc::new(RepartitionExec::try_new(coalesced_right, part_right)?); // 2d) Build an Inner HashJoinExec → left.value = right.value - let join: Arc = Arc::new(HashJoinExec::try_new( - hashed_left.clone(), - hashed_right.clone(), + let join = Arc::new(HashJoinExec::try_new( + hashed_left, + hashed_right, vec![( - Arc::new(Column::new_with_schema("value", &hashed_left.schema()).unwrap()), - Arc::new(Column::new_with_schema("value", &hashed_right.schema()).unwrap()), + Arc::new(Column::new_with_schema("value", &schema)?), + Arc::new(Column::new_with_schema("value", &schema)?), )], None, &JoinType::Inner, @@ -435,14 +593,13 @@ async fn test_infinite_join_agg_cancel() -> Result<(), Box> { // 3) Project only one column (“value” from the left side) because we just want to sum that let input_schema = join.schema(); - let proj_expr: Vec<(Arc, String)> = vec![( - Arc::new(Column::new_with_schema("value", &input_schema)?) - as Arc, + let proj_expr = vec![( + Arc::new(Column::new_with_schema("value", &input_schema)?) as _, "value".to_string(), )]; - let projection: Arc = - Arc::new(ProjectionExec::try_new(proj_expr, join.clone())?); + let projection = Arc::new(ProjectionExec::try_new(proj_expr, join)?); + let projection_schema = projection.schema(); let output_fields = vec![Field::new("total", DataType::Int64, true)]; let output_schema = Arc::new(Schema::new(output_fields)); @@ -455,17 +612,17 @@ async fn test_infinite_join_agg_cancel() -> Result<(), Box> { &projection.schema(), )?)], ) - .schema(output_schema.clone()) + .schema(output_schema) .alias("total") .build()?; - let aggr: Arc = Arc::new(AggregateExec::try_new( + let aggr = Arc::new(AggregateExec::try_new( AggregateMode::Single, PhysicalGroupBy::new(vec![], vec![], vec![]), vec![Arc::new(aggregate_expr)], vec![None], - projection.clone(), - projection.schema(), + projection, + projection_schema, )?); // 5) Wrap yields under each infinite leaf @@ -488,60 +645,59 @@ async fn test_infinite_join_agg_cancel() -> Result<(), Box> { Ok(()) } -// #[tokio::test] -// async fn test_filter_reject_all_batches_cancel() -> Result<(), Box> { -// // 1) Create a Session, Schema, and an 8K-row RecordBatch -// let session_ctx = SessionContext::new(); -// let schema: SchemaRef = Arc::new(Schema::new(vec![Field::new( -// "value", -// DataType::Int64, -// false, -// )])); -// -// // Build a batch with values 0..8191 -// let mut builder = Int64Array::builder(8_192); -// for v in 0..8_192 { -// builder.append_value(v); -// } -// let batch = Arc::new(RecordBatch::try_new( -// schema.clone(), -// vec![Arc::new(builder.finish())], -// )?); -// -// // 2a) Wrap this batch in an InfiniteExec -// let infinite = Arc::new(InfiniteExec::new(&batch)); -// -// // 2b) Construct a FilterExec that is always false: “value > 10000” (no rows pass) -// let false_predicate: Arc = Arc::new(BinaryExpr::new( -// Arc::new(Column::new_with_schema("value", &schema)?), -// Gt, -// Arc::new(Literal::new(ScalarValue::Int64(Some(10_000)))), -// )); -// let filtered: Arc = -// Arc::new(FilterExec::try_new(false_predicate, infinite.clone())?); -// -// // 2c) Use CoalesceBatchesExec to guarantee each Filter pull always yields an 8192-row batch -// let coalesced: Arc = -// Arc::new(CoalesceBatchesExec::new(filtered.clone(), 8_192)); -// -// // 3) WrapLeaves to insert YieldExec—so that the InfiniteExec yields control between batches -// let config = ConfigOptions::new(); -// let optimized = WrapLeaves::new().optimize(coalesced, &config)?; -// -// // 4) Execute with a 1-second timeout. Because Filter discards all 8192 rows each time -// // without ever producing output, no batch will arrive within 1 second. And since -// // emission type is not Final, we never see an end‐of‐stream marker. -// let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; -// const TIMEOUT: u64 = 1; -// let result = select! { -// batch_opt = stream.next() => batch_opt, -// _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { -// None -// } -// }; -// assert!( -// result.is_none(), -// "Expected no output for infinite + filter(all-false) + aggregate, but got a batch" -// ); -// Ok(()) -// } +#[rstest] +#[tokio::test] +#[ignore] +async fn test_filter_reject_all_batches_cancel( + #[values(false, true)] pretend_finite: bool, +) -> Result<(), Box> { + // 1) Create a Session, Schema, and an 8K-row RecordBatch + let session_ctx = SessionContext::new(); + let schema = Arc::new(Schema::new(vec![Field::new( + "value", + DataType::Int64, + false, + )])); + + // Build a batch with values 0..8191 + let mut builder = Int64Array::builder(8_192); + for v in 0..8_192 { + builder.append_value(v); + } + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(builder.finish())])?; + + // 2a) Wrap this batch in an InfiniteExec + let infinite = Arc::new(InfiniteExec::new(batch, pretend_finite)); + + // 2b) Construct a FilterExec that is always false: “value > 10000” (no rows pass) + let false_predicate = Arc::new(BinaryExpr::new( + Arc::new(Column::new_with_schema("value", &schema)?), + Gt, + Arc::new(Literal::new(ScalarValue::Int64(Some(10_000)))), + )); + let filtered = Arc::new(FilterExec::try_new(false_predicate, infinite)?); + + // 2c) Use CoalesceBatchesExec to guarantee each Filter pull always yields an 8192-row batch + let coalesced = Arc::new(CoalesceBatchesExec::new(filtered, 8_192)); + + // 3) WrapLeaves to insert YieldExec—so that the InfiniteExec yields control between batches + let config = ConfigOptions::new(); + let optimized = WrapLeaves::new().optimize(coalesced, &config)?; + + // 4) Execute with a 1-second timeout. Because Filter discards all 8192 rows each time + // without ever producing output, no batch will arrive within 1 second. And since + // emission type is not Final, we never see an end‐of‐stream marker. + let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; + const TIMEOUT: u64 = 1; + let result = select! { + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { + None + } + }; + assert!( + result.is_none(), + "Expected no output for infinite + filter(all-false) + aggregate, but got a batch" + ); + Ok(()) +} diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index b113fd63aa21..a99246655f85 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -20,21 +20,22 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::execution_plan::CardinalityEffect; -use crate::execution_plan::CardinalityEffect::Equal; +use crate::execution_plan::CardinalityEffect::{self, Equal}; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, SendableRecordBatchStream, }; use arrow::record_batch::RecordBatch; use arrow_schema::Schema; -use datafusion_common::{Result, Statistics}; +use datafusion_common::{internal_err, Result, Statistics}; use datafusion_execution::TaskContext; + use futures::{Stream, StreamExt}; -/// A stream that yields batches of data, yielding control back to the executor every `frequency` batches -/// -/// This can be useful to allow operators that might not yield to check for cancellation +/// An identity stream that passes batches through as is, but yields control +/// back to the runtime every `frequency` batches. This stream is useful to +/// construct a mechanism that allows operators that do not directly cooperate +/// with the runtime to check/support cancellation. pub struct YieldStream { inner: SendableRecordBatchStream, batches_processed: usize, @@ -51,7 +52,6 @@ impl YieldStream { } } -// Stream> to poll_next_unpin impl Stream for YieldStream { type Item = Result; @@ -65,43 +65,58 @@ impl Stream for YieldStream { return Poll::Pending; } - match self.inner.poll_next_unpin(cx) { - Poll::Ready(Some(Ok(batch))) => { + let value = self.inner.poll_next_unpin(cx); + match value { + Poll::Ready(Some(Ok(_))) => { self.batches_processed += 1; - Poll::Ready(Some(Ok(batch))) } Poll::Pending => { self.batches_processed = 0; - Poll::Pending } - - other => other, + _ => {} } + value } } -// RecordBatchStream schema() impl RecordBatchStream for YieldStream { fn schema(&self) -> Arc { self.inner.schema() } } +/// This operator wraps any other execution plan and to "adapt" it to cooperate +/// with the runtime by yielding control back to the runtime every `frequency` +/// batches. This is useful for operators that do not natively support yielding +/// control, allowing them to be used in a runtime that requires yielding for +/// cancellation or other purposes. #[derive(Debug)] pub struct YieldStreamExec { + /// The child execution plan that this operator "wraps" to make it + /// cooperate with the runtime. child: Arc, + /// The frequency at which the operator yields control back to the runtime. frequency: usize, - properties: PlanProperties, } impl YieldStreamExec { + /// Create a new `YieldStreamExec` operator that wraps the given child + /// execution plan and yields control back to the runtime every `frequency` + /// batches. pub fn new(child: Arc, frequency: usize) -> Self { - let properties = child.properties().clone(); - Self { - child, - properties, - frequency, - } + Self { frequency, child } + } + + /// Returns the child execution plan this operator "wraps" to make it + /// cooperate with the runtime. + pub fn input(&self) -> &Arc { + &self.child + } + + /// Returns the frequency at which the operator yields control back to the + /// runtime. + pub fn yield_frequency(&self) -> usize { + self.frequency } } @@ -115,18 +130,6 @@ impl DisplayAs for YieldStreamExec { } } -impl YieldStreamExec { - pub fn input(&self) -> &Arc { - &self.child - } -} - -impl YieldStreamExec { - pub fn get_yield_frequency(&self) -> usize { - self.frequency - } -} - impl ExecutionPlan for YieldStreamExec { fn name(&self) -> &str { "YieldStreamExec" @@ -141,7 +144,7 @@ impl ExecutionPlan for YieldStreamExec { } fn properties(&self) -> &PlanProperties { - &self.properties + self.child.properties() } fn children(&self) -> Vec<&Arc> { @@ -150,16 +153,13 @@ impl ExecutionPlan for YieldStreamExec { fn with_new_children( self: Arc, - children: Vec>, + mut children: Vec>, ) -> Result> { if children.len() != 1 { - return Err(datafusion_common::DataFusionError::Internal( - "YieldStreamExec requires exactly one child".to_string(), - )); + return internal_err!("YieldStreamExec requires exactly one child"); } - // Use Arc::clone on children[0] rather than calling clone() directly Ok(Arc::new(YieldStreamExec::new( - Arc::clone(&children[0]), + children.swap_remove(0), self.frequency, ))) } @@ -169,21 +169,16 @@ impl ExecutionPlan for YieldStreamExec { partition: usize, task_ctx: Arc, ) -> Result { - let child_stream = self.child.execute(partition, Arc::clone(&task_ctx))?; + let child_stream = self.child.execute(partition, task_ctx)?; let yield_stream = YieldStream::new(child_stream, self.frequency); Ok(Box::pin(yield_stream)) } fn partition_statistics(&self, partition: Option) -> Result { - if partition.is_none() { - self.child.partition_statistics(partition) - } else { - Ok(Statistics::new_unknown(&self.schema())) - } + self.child.partition_statistics(partition) } fn maintains_input_order(&self) -> Vec { - // YieldStreamExec does not change the order of the input data self.child.maintains_input_order() } @@ -200,7 +195,9 @@ impl ExecutionPlan for YieldStreamExec { mod tests { use super::*; use crate::stream::RecordBatchStreamAdapter; + use arrow_schema::SchemaRef; + use futures::{stream, StreamExt}; // Frequency testing: @@ -224,7 +221,7 @@ mod tests { async fn yield_less_than_threshold() -> Result<()> { let count = YIELD_BATCHES - 10; let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner, YIELD_BATCHES) + let out = YieldStream::new(inner, YIELD_BATCHES) .collect::>() .await; assert_eq!(out.len(), count); @@ -235,7 +232,7 @@ mod tests { async fn yield_equal_to_threshold() -> Result<()> { let count = YIELD_BATCHES; let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner, YIELD_BATCHES) + let out = YieldStream::new(inner, YIELD_BATCHES) .collect::>() .await; assert_eq!(out.len(), count); @@ -246,7 +243,7 @@ mod tests { async fn yield_more_than_threshold() -> Result<()> { let count = YIELD_BATCHES + 20; let inner = make_empty_batches(count); - let out: Vec<_> = YieldStream::new(inner, YIELD_BATCHES) + let out = YieldStream::new(inner, YIELD_BATCHES) .collect::>() .await; assert_eq!(out.len(), count); diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 89ac3cc56f6c..a6037a400145 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -2787,7 +2787,7 @@ impl protobuf::PhysicalPlanNode { extension_codec, )?; - let frequency = exec.get_yield_frequency(); + let frequency = exec.yield_frequency(); Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::YieldStream(Box::new( From b05cb85e683604a7cb6219f70281bcdba5654fcb Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 21:50:12 +0800 Subject: [PATCH 56/74] Add remaining test --- .../core/tests/execution/infinite_cancel.rs | 76 +++++++++++++++++++ 1 file changed, 76 insertions(+) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index 83fdde278555..083b2f95f8a9 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -701,3 +701,79 @@ async fn test_filter_reject_all_batches_cancel( ); Ok(()) } + +#[rstest] +#[tokio::test] +#[ignore] +async fn test_infinite_hash_join_without_repartition_and_no_agg( + #[values(false, true)] pretend_finite: bool, +) -> Result<(), Box> { + // 1) Create Session, schema, and an 8K-row RecordBatch for each side + let session_ctx = SessionContext::new(); + let schema: SchemaRef = Arc::new(Schema::new(vec![Field::new( + "value", + DataType::Int64, + false, + )])); + + let mut builder_left = Int64Array::builder(8_192); + let mut builder_right = Int64Array::builder(8_192); + for v in 0..8_192 { + builder_left.append_value(v); + builder_right.append_value(v + 1); + } + let batch_left = + RecordBatch::try_new(schema.clone(), vec![Arc::new(builder_left.finish())])?; + let batch_right = + RecordBatch::try_new(schema.clone(), vec![Arc::new(builder_right.finish())])?; + + // 2a) Unlike the test with aggregation, keep this as a pure join— + // use InfiniteExec to simulate an infinite stream + let infinite_left = Arc::new(InfiniteExec::new(batch_left, pretend_finite)); + let infinite_right = Arc::new(InfiniteExec::new(batch_right, pretend_finite)); + + // 2b) To feed a single batch into the Join, we can still use CoalesceBatchesExec, + // but do NOT wrap it in a RepartitionExec + let coalesced_left = Arc::new(CoalesceBatchesExec::new(infinite_left, 8_192)); + let coalesced_right = Arc::new(CoalesceBatchesExec::new(infinite_right, 8_192)); + + // 2c) Directly feed `coalesced_left` and `coalesced_right` into HashJoinExec. + // Do not use aggregation or repartition. + let join = Arc::new(HashJoinExec::try_new( + coalesced_left.clone(), + coalesced_right.clone(), + vec![( + Arc::new(Column::new_with_schema("value", &schema)?), + Arc::new(Column::new_with_schema("value", &schema)?), + )], + /* filter */ None, + &JoinType::Inner, + /* output64 */ None, + // Using CollectLeft is fine—just avoid RepartitionExec’s partitioned channels. + PartitionMode::CollectLeft, + /* build_left */ true, + )?); + + // 3) Do not apply WrapLeaves—since there is no aggregation, WrapLeaves would + // not insert a 'final' yield wrapper for the Join. If you want to skip WrapLeaves + // entirely, comment out the next line; however, not calling it is equivalent + // because there is no aggregation so no wrapper is inserted. Here we simply do + // not call WrapLeaves, ensuring the plan has neither aggregation nor repartition. + let optimized = join as Arc; + + // 4) Execute with a 1 second timeout + let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; + const TIMEOUT_SEC: u64 = 1; + let result = select! { + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT_SEC)) => { + None + } + }; + + assert!( + result.is_none(), + "Expected no output for infinite + filter(all-false) + aggregate, but got a batch" + ); + Ok(()) +} From e136a0368e73f0cceb283d4de700d633ca3a2bb8 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 22:07:12 +0800 Subject: [PATCH 57/74] Add sort merge case --- .../core/tests/execution/infinite_cancel.rs | 84 ++++++++++++++++++- 1 file changed, 81 insertions(+), 3 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index 083b2f95f8a9..362cb8ed5aa7 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -22,7 +22,7 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use arrow::array::{Int64Array, RecordBatch}; +use arrow::array::{Array, Int64Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use arrow_schema::SortOptions; use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; @@ -41,14 +41,14 @@ use datafusion::{common, physical_plan}; use datafusion_common::config::ConfigOptions; use datafusion_common::{JoinType, ScalarValue}; use datafusion_expr_common::operator::Operator::Gt; -use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; +use datafusion_physical_expr::expressions::{col, BinaryExpr, Column, Literal}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use datafusion_physical_optimizer::wrap_leaves_cancellation::WrapLeaves; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::filter::FilterExec; -use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; +use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; @@ -777,3 +777,81 @@ async fn test_infinite_hash_join_without_repartition_and_no_agg( ); Ok(()) } + +#[rstest] +#[tokio::test] +#[ignore] +async fn test_infinite_sort_merge_join_without_repartition_and_no_agg( + #[values(false, true)] pretend_finite: bool, +) -> Result<(), Box> { + // 1) Create Session, schema, and two small RecordBatches that never overlap: + // Left = [-3, -2, -1], Right = [0, 1, 2] + let session_ctx = SessionContext::new(); + let schema: SchemaRef = Arc::new(Schema::new(vec![Field::new( + "value", + DataType::Int64, + false, + )])); + + let left_array = { + let mut b = Int64Array::builder(3); + b.append_value(-3); + b.append_value(-2); + b.append_value(-1); + Arc::new(b.finish()) as Arc + }; + let right_array = { + let mut b = Int64Array::builder(3); + b.append_value(0); + b.append_value(1); + b.append_value(2); + Arc::new(b.finish()) as Arc + }; + let batch_left = RecordBatch::try_new(schema.clone(), vec![left_array])?; + let batch_right = RecordBatch::try_new(schema.clone(), vec![right_array])?; + + // 2a) Wrap each small batch in an InfiniteExec (pretend_finite toggles finite vs infinite) + let infinite_left = Arc::new(InfiniteExec::new(batch_left, pretend_finite)); + let infinite_right = Arc::new(InfiniteExec::new(batch_right, pretend_finite)); + + // 2b) Coalesce each InfiniteExec into a single 3-row batch at a time. + // (Do NOT wrap in RepartitionExec.) + let coalesced_left = Arc::new(CoalesceBatchesExec::new(infinite_left, 3)); + let coalesced_right = Arc::new(CoalesceBatchesExec::new(infinite_right, 3)); + + // 2c) Build a SortMergeJoinExec on “value”. Since left values < 0 and + // right values ≥ 0, they never match. No aggregation or repartition. + // + // We need a Vec for the join key. Any consistent SortOptions works, + // because data is already in ascending order on “value.” + let join = Arc::new(SortMergeJoinExec::try_new( + coalesced_left.clone(), + coalesced_right.clone(), + vec![( + col("value", &coalesced_left.schema())?, + col("value", &coalesced_right.schema())?, + )], + /* filter */ None, + JoinType::Inner, + vec![SortOptions::new(true, false)], // ascending, nulls last + /* null_equal */ true, + )?); + + // 3) Do not apply WrapLeaves (no aggregation, no repartition → no built-in yields). + let optimized = join as Arc; + + // 4) Execute with a 1-second timeout. Because both sides are infinite and never match, + // the SortMergeJoin will never produce output within 1s. + let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; + const TIMEOUT_SEC: u64 = 1; + let result = select! { + batch_opt = stream.next() => batch_opt, + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT_SEC)) => None, + }; + + assert!( + result.is_none(), + "Expected no output for infinite SortMergeJoin (no repartition & no aggregation), but got a batch", + ); + Ok(()) +} From 29bc2c77b9bab9aadb039eb5647c6ab8ef0d6997 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 22:16:51 +0800 Subject: [PATCH 58/74] change rule solution --- .../core/tests/execution/infinite_cancel.rs | 4 - .../src/wrap_leaves_cancellation.rs | 72 +++---- .../sqllogictest/test_files/aggregate.slt | 61 ++++-- datafusion/sqllogictest/test_files/array.slt | 6 +- .../test_files/count_star_rule.slt | 3 +- datafusion/sqllogictest/test_files/cte.slt | 85 ++++---- datafusion/sqllogictest/test_files/ddl.slt | 4 +- .../sqllogictest/test_files/explain.slt | 15 +- .../sqllogictest/test_files/explain_tree.slt | 201 ++++++++++++------ .../test_files/filter_without_sort_exec.slt | 18 +- .../sqllogictest/test_files/group_by.slt | 30 ++- .../sqllogictest/test_files/join.slt.part | 4 +- datafusion/sqllogictest/test_files/joins.slt | 30 +-- datafusion/sqllogictest/test_files/limit.slt | 16 +- datafusion/sqllogictest/test_files/order.slt | 8 +- .../sqllogictest/test_files/predicates.slt | 8 +- .../sqllogictest/test_files/repartition.slt | 3 +- .../sqllogictest/test_files/subquery.slt | 8 +- datafusion/sqllogictest/test_files/union.slt | 13 +- datafusion/sqllogictest/test_files/window.slt | 21 +- 20 files changed, 371 insertions(+), 239 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index 362cb8ed5aa7..d10948249357 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -267,7 +267,6 @@ async fn test_infinite_sort_cancel( #[rstest] #[tokio::test] -#[ignore] async fn test_infinite_interleave_cancel( #[values(false, true)] pretend_finite: bool, ) -> Result<(), Box> { @@ -647,7 +646,6 @@ async fn test_infinite_join_agg_cancel( #[rstest] #[tokio::test] -#[ignore] async fn test_filter_reject_all_batches_cancel( #[values(false, true)] pretend_finite: bool, ) -> Result<(), Box> { @@ -704,7 +702,6 @@ async fn test_filter_reject_all_batches_cancel( #[rstest] #[tokio::test] -#[ignore] async fn test_infinite_hash_join_without_repartition_and_no_agg( #[values(false, true)] pretend_finite: bool, ) -> Result<(), Box> { @@ -780,7 +777,6 @@ async fn test_infinite_hash_join_without_repartition_and_no_agg( #[rstest] #[tokio::test] -#[ignore] async fn test_infinite_sort_merge_join_without_repartition_and_no_agg( #[values(false, true)] pretend_finite: bool, ) -> Result<(), Box> { diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index 06325d6c4ba8..ca25a401801b 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -19,15 +19,18 @@ use crate::PhysicalOptimizerRule; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion}; use datafusion_common::Result; -use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::ExecutionPlan; use std::fmt::{Debug, Formatter}; use std::sync::Arc; -/// WrapLeaves is a PhysicalOptimizerRule that finds every -/// pipeline‐breaking node (emission_type == Final) and then -/// wraps all of its leaf children in YieldStreamExec. +/// WrapLeaves is a PhysicalOptimizerRule that finds every *leaf* node in the +/// entire plan, and replaces it with a variant that can cooperatively yield +/// (either using its built‐in `with_cooperative_yields()` or, if none exists, +/// by wrapping it in a `YieldStreamExec` wrapper). +/// +/// In contrast to the previous behavior (which only looked at “Final”/pipeline‐ +/// breaking nodes), this modified rule simply wraps *every* leaf no matter what. pub struct WrapLeaves {} impl WrapLeaves { @@ -35,60 +38,37 @@ impl WrapLeaves { Self {} } - /// This function is called on every plan node during transform_down(). - /// If the node is a leaf (no children), we wrap it in a new YieldStreamExec - /// and stop recursing further under that branch (TreeNodeRecursion::Jump). + /// Called when we encounter any node during `transform_down()`. If the node + /// has no children, it is a leaf. We check if it has a built‐in cooperative + /// yield variant (`with_cooperative_yields()`); if so, we replace it with that. + /// Otherwise, we wrap it in a `YieldStreamExec`. + /// + /// We then return `TreeNodeRecursion::Jump` so that we do not attempt to go + /// deeper under this node (there are no children, anyway). fn wrap_leaves( plan: Arc, yield_frequency: usize, ) -> Result>> { if plan.children().is_empty() { - // If the leaf node already has a built-in yielding variant: + // This is a leaf. Try to see if the plan itself has a cooperative‐yield variant. if let Some(coop_variant) = Arc::clone(&plan).with_cooperative_yields() { - // Replace it with the built-in yielding version. + // Replace with the built‐in cooperative yield version. Ok(Transformed::new( coop_variant, - /* changed = */ true, + /* changed= */ true, TreeNodeRecursion::Jump, )) } else { - // Otherwise, wrap in a YieldStreamExec. + // Otherwise wrap it in a YieldStreamExec to enforce periodic yielding. let wrapped = Arc::new(YieldStreamExec::new(plan, yield_frequency)); Ok(Transformed::new( wrapped, - /* changed = */ true, + /* changed= */ true, TreeNodeRecursion::Jump, )) } } else { - // Not a leaf: leave unchanged and keep recursing - Ok(Transformed::no(plan)) - } - } - - /// This function is called on every plan node during transform_down(). - /// - /// If this node itself is a pipeline breaker (emission_type == Final), - /// we perform a second pass of transform_down with wrap_leaves. Then we - /// set TreeNodeRecursion::Jump so that we do not descend any deeper under - /// this subtree (we’ve already wrapped its leaves). - fn wrap_leaves_of_pipeline_breakers( - plan: Arc, - yield_frequency: usize, - ) -> Result>> { - let is_pipeline_breaker = plan.properties().emission_type == EmissionType::Final; - if is_pipeline_breaker { - // Transform all leaf descendants of this node by calling wrap_leaves - let mut transformed = - plan.transform_down(|child_plan: Arc| { - Self::wrap_leaves(child_plan, yield_frequency) - })?; - - // Once we’ve handled the leaves of this subtree, we skip deeper recursion - transformed.tnr = TreeNodeRecursion::Jump; - Ok(transformed) - } else { - // Not a pipeline breaker: do nothing here, let transform_down recurse + // Not a leaf: leave unchanged for now, keep recursing down. Ok(Transformed::no(plan)) } } @@ -116,23 +96,27 @@ impl PhysicalOptimizerRule for WrapLeaves { plan: Arc, config: &ConfigOptions, ) -> Result> { + // Only activate if user has configured a nonzero yield frequency. if config.optimizer.yield_frequency_for_pipeline_break != 0 { let yield_frequency = config.optimizer.yield_frequency_for_pipeline_break; - // We run a top‐level transform_down: for every node, call wrap_leaves_of_pipeline_breakers. - // If a node is a pipeline breaker, we then wrap all of its leaf children in YieldStreamExec. + // We perform a single top‐level transform_down over the entire plan. + // For each node encountered, we call `wrap_leaves`. If the node is + // a leaf, it will be replaced with a yielding variant (either its + // built‐in cooperative version or an explicit YieldStreamExec). let new_plan = plan.transform_down(|node: Arc| { - Self::wrap_leaves_of_pipeline_breakers(node, yield_frequency) + Self::wrap_leaves(node, yield_frequency) })?; Ok(new_plan.data) } else { + // If yield_frequency is zero, we do nothing. Ok(plan) } } fn schema_check(&self) -> bool { - // Wrapping a leaf in YieldStreamExec preserves the schema, so we’re fine + // Wrapping a leaf in YieldStreamExec preserves the schema, so it is safe. true } } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index ed77435d6a85..eaa627a8747d 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -380,15 +380,20 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[array_agg(DISTINCT a.foo), sum(DISTINCT Int64(1))], ordering_mode=Sorted 06)----------UnionExec 07)------------ProjectionExec: expr=[1 as id, 2 as foo] -08)--------------PlaceholderRowExec -09)------------ProjectionExec: expr=[1 as id, NULL as foo] -10)--------------PlaceholderRowExec -11)------------ProjectionExec: expr=[1 as id, NULL as foo] -12)--------------PlaceholderRowExec -13)------------ProjectionExec: expr=[1 as id, 3 as foo] -14)--------------PlaceholderRowExec -15)------------ProjectionExec: expr=[1 as id, 2 as foo] -16)--------------PlaceholderRowExec +08)--------------YieldStreamExec frequency=64 +09)----------------PlaceholderRowExec +10)------------ProjectionExec: expr=[1 as id, NULL as foo] +11)--------------YieldStreamExec frequency=64 +12)----------------PlaceholderRowExec +13)------------ProjectionExec: expr=[1 as id, NULL as foo] +14)--------------YieldStreamExec frequency=64 +15)----------------PlaceholderRowExec +16)------------ProjectionExec: expr=[1 as id, 3 as foo] +17)--------------YieldStreamExec frequency=64 +18)----------------PlaceholderRowExec +19)------------ProjectionExec: expr=[1 as id, 2 as foo] +20)--------------YieldStreamExec frequency=64 +21)----------------PlaceholderRowExec # FIX: custom absolute values @@ -6067,7 +6072,8 @@ logical_plan 02)--TableScan: empty projection=[col0] physical_plan 01)ProjectionExec: expr=[NULL as min(empty.col0)] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query TT EXPLAIN SELECT MAX(col0) FROM empty; @@ -6077,7 +6083,8 @@ logical_plan 02)--TableScan: empty projection=[col0] physical_plan 01)ProjectionExec: expr=[NULL as max(empty.col0)] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec statement ok DROP TABLE empty; @@ -6534,15 +6541,20 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[last_value(a.foo) ORDER BY [a.foo ASC NULLS LAST], sum(DISTINCT Int64(1))], ordering_mode=Sorted 06)----------UnionExec 07)------------ProjectionExec: expr=[1 as id, 2 as foo] -08)--------------PlaceholderRowExec -09)------------ProjectionExec: expr=[1 as id, 4 as foo] -10)--------------PlaceholderRowExec -11)------------ProjectionExec: expr=[1 as id, 5 as foo] -12)--------------PlaceholderRowExec -13)------------ProjectionExec: expr=[1 as id, 3 as foo] -14)--------------PlaceholderRowExec -15)------------ProjectionExec: expr=[1 as id, 2 as foo] -16)--------------PlaceholderRowExec +08)--------------YieldStreamExec frequency=64 +09)----------------PlaceholderRowExec +10)------------ProjectionExec: expr=[1 as id, 4 as foo] +11)--------------YieldStreamExec frequency=64 +12)----------------PlaceholderRowExec +13)------------ProjectionExec: expr=[1 as id, 5 as foo] +14)--------------YieldStreamExec frequency=64 +15)----------------PlaceholderRowExec +16)------------ProjectionExec: expr=[1 as id, 3 as foo] +17)--------------YieldStreamExec frequency=64 +18)----------------PlaceholderRowExec +19)------------ProjectionExec: expr=[1 as id, 2 as foo] +20)--------------YieldStreamExec frequency=64 +21)----------------PlaceholderRowExec # SortExec is removed if it is coming after one-row producing AggregateExec's having an empty group by expression query TT @@ -6672,7 +6684,8 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(Int64(1)), 2 as count()] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query II select count(1), count(*) from t; @@ -6688,7 +6701,8 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(Int64(1)), 2 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query II select count(), count(*) from t; @@ -6704,7 +6718,8 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(), 2 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query TT explain select count(1) * count(2) from t; diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index d89ba600d7a6..5b23d94c4725 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -7196,7 +7196,8 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[true as array_has_all(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(1),Int64(3)))] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec # array containment operator with scalars #2 (arrow at) query BBBBBBB @@ -7219,7 +7220,8 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[true as array_has_all(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(1),Int64(3)))] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec ### Array casting tests diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index d38d3490fed4..4d0b92cd7d04 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -37,7 +37,8 @@ logical_plan 04)------EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as count()] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query TT EXPLAIN SELECT t1.a, COUNT() FROM t1 GROUP BY t1.a; diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index d3267d772553..4364688cac66 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -36,7 +36,8 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec # cte_use_same_name_multiple_times statement error DataFusion error: Error during planning: WITH query name "a" specified more than once @@ -114,13 +115,15 @@ logical_plan physical_plan 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 +03)----YieldStreamExec frequency=64 +04)------PlaceholderRowExec +05)--CoalescePartitionsExec +06)----ProjectionExec: expr=[id@0 + 1 as id] +07)------CoalesceBatchesExec: target_batch_size=8192 +08)--------FilterExec: id@0 < 10 +09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------YieldStreamExec frequency=64 +11)--------------WorkTableExec: name=nodes # setup statement ok @@ -732,18 +735,19 @@ logical_plan physical_plan 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)--------------YieldStreamExec frequency=64 -11)----------------WorkTableExec: name=recursive_cte -12)------ProjectionExec: expr=[2 as val] -13)--------YieldStreamExec frequency=64 -14)----------PlaceholderRowExec +03)----YieldStreamExec frequency=64 +04)------PlaceholderRowExec +05)--ProjectionExec: expr=[2 as val] +06)----CrossJoinExec +07)------CoalescePartitionsExec +08)--------CoalesceBatchesExec: target_batch_size=8182 +09)----------FilterExec: val@0 < 2 +10)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------------YieldStreamExec frequency=64 +12)----------------WorkTableExec: name=recursive_cte +13)------ProjectionExec: expr=[2 as val] +14)--------YieldStreamExec frequency=64 +15)----------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/9794 # Non-recursive term and recursive term have different types @@ -875,7 +879,8 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query TT explain with numbers(a,b,c) as (select 1,2,3) select * from numbers; @@ -886,7 +891,8 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query TT explain with numbers as (select 1 as a, 2 as b, 3 as c) select * from numbers; @@ -897,7 +903,8 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec statement count 0 create table person (id int, name string, primary key(id)) @@ -966,13 +973,15 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=numbers, is_distinct=false 02)--ProjectionExec: expr=[1 as n] -03)----PlaceholderRowExec -04)--CoalescePartitionsExec -05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] -06)------CoalesceBatchesExec: target_batch_size=8182 -07)--------FilterExec: n@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)------------WorkTableExec: name=numbers +03)----YieldStreamExec frequency=64 +04)------PlaceholderRowExec +05)--CoalescePartitionsExec +06)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] +07)------CoalesceBatchesExec: target_batch_size=8182 +08)--------FilterExec: n@0 < 10 +09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------YieldStreamExec frequency=64 +11)--------------WorkTableExec: name=numbers query TT explain WITH RECURSIVE numbers AS ( @@ -992,13 +1001,15 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=numbers, is_distinct=false 02)--ProjectionExec: expr=[1 as n] -03)----PlaceholderRowExec -04)--CoalescePartitionsExec -05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] -06)------CoalesceBatchesExec: target_batch_size=8182 -07)--------FilterExec: n@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)------------WorkTableExec: name=numbers +03)----YieldStreamExec frequency=64 +04)------PlaceholderRowExec +05)--CoalescePartitionsExec +06)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] +07)------CoalesceBatchesExec: target_batch_size=8182 +08)--------FilterExec: n@0 < 10 +09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------YieldStreamExec frequency=64 +11)--------------WorkTableExec: name=numbers statement count 0 set datafusion.execution.enable_recursive_ctes = false; diff --git a/datafusion/sqllogictest/test_files/ddl.slt b/datafusion/sqllogictest/test_files/ddl.slt index 1e95e426f3e0..8ba9b0896c43 100644 --- a/datafusion/sqllogictest/test_files/ddl.slt +++ b/datafusion/sqllogictest/test_files/ddl.slt @@ -713,7 +713,9 @@ query TT explain select c1 from t; ---- logical_plan TableScan: t projection=[c1] -physical_plan StreamingTableExec: partition_sizes=1, projection=[c1], infinite_source=true +physical_plan +01)YieldStreamExec frequency=64 +02)--StreamingTableExec: partition_sizes=1, projection=[c1], infinite_source=true statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 8c9054d56c14..b20c256a9801 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -89,7 +89,8 @@ EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c ---- physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec statement ok set datafusion.explain.physical_plan_only = false @@ -385,7 +386,8 @@ logical_plan 02)--EmptyRelation physical_plan 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 +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query TT explain select [[1, 2, 3], [4, 5, 6]]; @@ -395,7 +397,8 @@ logical_plan 02)--EmptyRelation physical_plan 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 +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec # Explain Struct @@ -407,7 +410,8 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[{c0:1,c1:2.3,c2:abc} as struct(Int64(1),Float64(2.3),Utf8("abc"))] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec statement ok @@ -430,7 +434,8 @@ physical_plan 01)NestedLoopJoinExec: join_type=LeftSemi 02)--DataSourceExec: partitions=1, partition_sizes=[0] 03)--ProjectionExec: expr=[] -04)----PlaceholderRowExec +04)----YieldStreamExec frequency=64 +05)------PlaceholderRowExec statement ok drop table t1; diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 15bf61576571..77ddf4208927 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -1151,11 +1151,17 @@ physical_plan 06)│ ASC NULLS LAST │ 07)└─────────────┬─────────────┘ 08)┌─────────────┴─────────────┐ -09)│ StreamingTableExec │ +09)│ YieldStreamExec │ 10)│ -------------------- │ -11)│ infinite: true │ -12)│ limit: None │ -13)└───────────────────────────┘ +11)│ YieldStreamExec frequency:│ +12)│ 64 │ +13)└─────────────┬─────────────┘ +14)┌─────────────┴─────────────┐ +15)│ StreamingTableExec │ +16)│ -------------------- │ +17)│ infinite: true │ +18)│ limit: None │ +19)└───────────────────────────┘ query TT EXPLAIN SELECT * @@ -1174,11 +1180,17 @@ physical_plan 08)│ limit: 50 │ 09)└─────────────┬─────────────┘ 10)┌─────────────┴─────────────┐ -11)│ StreamingTableExec │ +11)│ YieldStreamExec │ 12)│ -------------------- │ -13)│ infinite: true │ -14)│ limit: None │ -15)└───────────────────────────┘ +13)│ YieldStreamExec frequency:│ +14)│ 64 │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ StreamingTableExec │ +18)│ -------------------- │ +19)│ infinite: true │ +20)│ limit: None │ +21)└───────────────────────────┘ # Query with hash join. query TT @@ -1613,11 +1625,17 @@ physical_plan 25)│ RoundRobinBatch(4) │ 26)└─────────────┬─────────────┘ 27)┌─────────────┴─────────────┐ -28)│ StreamingTableExec │ +28)│ YieldStreamExec │ 29)│ -------------------- │ -30)│ infinite: true │ -31)│ limit: None │ -32)└───────────────────────────┘ +30)│ YieldStreamExec frequency:│ +31)│ 64 │ +32)└─────────────┬─────────────┘ +33)┌─────────────┴─────────────┐ +34)│ StreamingTableExec │ +35)│ -------------------- │ +36)│ infinite: true │ +37)│ limit: None │ +38)└───────────────────────────┘ # constant ticker, CAST(time AS DATE) = time, order by time @@ -1655,11 +1673,17 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ +29)│ YieldStreamExec │ 30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +31)│ YieldStreamExec frequency:│ +32)│ 64 │ +33)└─────────────┬─────────────┘ +34)┌─────────────┴─────────────┐ +35)│ StreamingTableExec │ +36)│ -------------------- │ +37)│ infinite: true │ +38)│ limit: None │ +39)└───────────────────────────┘ # same thing but order by date query TT @@ -1696,11 +1720,17 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ +29)│ YieldStreamExec │ 30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +31)│ YieldStreamExec frequency:│ +32)│ 64 │ +33)└─────────────┬─────────────┘ +34)┌─────────────┴─────────────┐ +35)│ StreamingTableExec │ +36)│ -------------------- │ +37)│ infinite: true │ +38)│ limit: None │ +39)└───────────────────────────┘ # same thing but order by ticker query TT @@ -1735,11 +1765,17 @@ physical_plan 24)│ RoundRobinBatch(4) │ 25)└─────────────┬─────────────┘ 26)┌─────────────┴─────────────┐ -27)│ StreamingTableExec │ +27)│ YieldStreamExec │ 28)│ -------------------- │ -29)│ infinite: true │ -30)│ limit: None │ -31)└───────────────────────────┘ +29)│ YieldStreamExec frequency:│ +30)│ 64 │ +31)└─────────────┬─────────────┘ +32)┌─────────────┴─────────────┐ +33)│ StreamingTableExec │ +34)│ -------------------- │ +35)│ infinite: true │ +36)│ limit: None │ +37)└───────────────────────────┘ # same thing but order by time, date @@ -1778,11 +1814,17 @@ physical_plan 27)│ RoundRobinBatch(4) │ 28)└─────────────┬─────────────┘ 29)┌─────────────┴─────────────┐ -30)│ StreamingTableExec │ +30)│ YieldStreamExec │ 31)│ -------------------- │ -32)│ infinite: true │ -33)│ limit: None │ -34)└───────────────────────────┘ +32)│ YieldStreamExec frequency:│ +33)│ 64 │ +34)└─────────────┬─────────────┘ +35)┌─────────────┴─────────────┐ +36)│ StreamingTableExec │ +37)│ -------------------- │ +38)│ infinite: true │ +39)│ limit: None │ +40)└───────────────────────────┘ @@ -1822,11 +1864,17 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ +29)│ YieldStreamExec │ 30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +31)│ YieldStreamExec frequency:│ +32)│ 64 │ +33)└─────────────┬─────────────┘ +34)┌─────────────┴─────────────┐ +35)│ StreamingTableExec │ +36)│ -------------------- │ +37)│ infinite: true │ +38)│ limit: None │ +39)└───────────────────────────┘ @@ -1851,35 +1899,42 @@ physical_plan 07)│ id: 1 ││ │ 08)└─────────────┬─────────────┘└─────────────┬─────────────┘ 09)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -10)│ PlaceholderRowExec ││ ProjectionExec │ -11)│ ││ -------------------- │ -12)│ ││ id: id + 1 │ -13)└───────────────────────────┘└─────────────┬─────────────┘ -14)-----------------------------┌─────────────┴─────────────┐ -15)-----------------------------│ CoalesceBatchesExec │ -16)-----------------------------│ -------------------- │ -17)-----------------------------│ target_batch_size: │ -18)-----------------------------│ 8192 │ -19)-----------------------------└─────────────┬─────────────┘ -20)-----------------------------┌─────────────┴─────────────┐ -21)-----------------------------│ FilterExec │ -22)-----------------------------│ -------------------- │ -23)-----------------------------│ predicate: id < 10 │ -24)-----------------------------└─────────────┬─────────────┘ -25)-----------------------------┌─────────────┴─────────────┐ -26)-----------------------------│ RepartitionExec │ -27)-----------------------------│ -------------------- │ -28)-----------------------------│ partition_count(in->out): │ -29)-----------------------------│ 1 -> 4 │ -30)-----------------------------│ │ -31)-----------------------------│ partitioning_scheme: │ -32)-----------------------------│ RoundRobinBatch(4) │ -33)-----------------------------└─────────────┬─────────────┘ -34)-----------------------------┌─────────────┴─────────────┐ -35)-----------------------------│ WorkTableExec │ -36)-----------------------------│ -------------------- │ -37)-----------------------------│ name: nodes │ -38)-----------------------------└───────────────────────────┘ +10)│ YieldStreamExec ││ ProjectionExec │ +11)│ -------------------- ││ -------------------- │ +12)│ YieldStreamExec frequency:││ id: id + 1 │ +13)│ 64 ││ │ +14)└─────────────┬─────────────┘└─────────────┬─────────────┘ +15)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +16)│ PlaceholderRowExec ││ CoalesceBatchesExec │ +17)│ ││ -------------------- │ +18)│ ││ target_batch_size: │ +19)│ ││ 8192 │ +20)└───────────────────────────┘└─────────────┬─────────────┘ +21)-----------------------------┌─────────────┴─────────────┐ +22)-----------------------------│ FilterExec │ +23)-----------------------------│ -------------------- │ +24)-----------------------------│ predicate: id < 10 │ +25)-----------------------------└─────────────┬─────────────┘ +26)-----------------------------┌─────────────┴─────────────┐ +27)-----------------------------│ RepartitionExec │ +28)-----------------------------│ -------------------- │ +29)-----------------------------│ partition_count(in->out): │ +30)-----------------------------│ 1 -> 4 │ +31)-----------------------------│ │ +32)-----------------------------│ partitioning_scheme: │ +33)-----------------------------│ RoundRobinBatch(4) │ +34)-----------------------------└─────────────┬─────────────┘ +35)-----------------------------┌─────────────┴─────────────┐ +36)-----------------------------│ YieldStreamExec │ +37)-----------------------------│ -------------------- │ +38)-----------------------------│ YieldStreamExec frequency:│ +39)-----------------------------│ 64 │ +40)-----------------------------└─────────────┬─────────────┘ +41)-----------------------------┌─────────────┴─────────────┐ +42)-----------------------------│ WorkTableExec │ +43)-----------------------------│ -------------------- │ +44)-----------------------------│ name: nodes │ +45)-----------------------------└───────────────────────────┘ query TT explain COPY (VALUES (1, 'foo', 1, '2023-01-01'), (2, 'bar', 2, '2023-01-02'), (3, 'baz', 3, '2023-01-03')) @@ -2069,11 +2124,17 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ +29)│ YieldStreamExec │ 30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +31)│ YieldStreamExec frequency:│ +32)│ 64 │ +33)└─────────────┬─────────────┘ +34)┌─────────────┴─────────────┐ +35)│ StreamingTableExec │ +36)│ -------------------- │ +37)│ infinite: true │ +38)│ limit: None │ +39)└───────────────────────────┘ # Test explain tree for PlaceholderRowExec query TT @@ -2086,5 +2147,11 @@ physical_plan 04)│ count(*): 2 │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ PlaceholderRowExec │ -08)└───────────────────────────┘ +07)│ YieldStreamExec │ +08)│ -------------------- │ +09)│ YieldStreamExec frequency:│ +10)│ 64 │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ PlaceholderRowExec │ +14)└───────────────────────────┘ diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index a09d8ce26ddf..6e74466bd10c 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -41,7 +41,8 @@ physical_plan 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] +05)--------YieldStreamExec frequency=64 +06)----------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 @@ -58,7 +59,8 @@ physical_plan 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] +05)--------YieldStreamExec frequency=64 +06)----------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 @@ -75,7 +77,8 @@ physical_plan 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] +05)--------YieldStreamExec frequency=64 +06)----------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 @@ -92,7 +95,8 @@ physical_plan 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] +05)--------YieldStreamExec frequency=64 +06)----------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 @@ -109,7 +113,8 @@ physical_plan 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] +05)--------YieldStreamExec frequency=64 +06)----------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 @@ -150,4 +155,5 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: date@0 = 2006-01-02 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] +05)--------YieldStreamExec frequency=64 +06)----------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 9e67018ecd0b..6b030586afdb 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2112,7 +2112,8 @@ logical_plan physical_plan 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] +03)----YieldStreamExec frequency=64 +04)------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 @@ -2143,7 +2144,8 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=PartiallySorted([1]) -03)----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] +03)----YieldStreamExec frequency=64 +04)------StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] query III SELECT a, d, @@ -2176,7 +2178,8 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], 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] +03)----YieldStreamExec frequency=64 +04)------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 @@ -2202,7 +2205,8 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], 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] +03)----YieldStreamExec frequency=64 +04)------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 @@ -2229,7 +2233,8 @@ logical_plan physical_plan 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] +03)----YieldStreamExec frequency=64 +04)------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 c) as last_c @@ -2251,7 +2256,8 @@ logical_plan 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan 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] +02)--YieldStreamExec frequency=64 +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 * @@ -2264,7 +2270,8 @@ logical_plan 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan 01)PartialSortExec: TopK(fetch=50), 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] +02)--YieldStreamExec frequency=64 +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 * @@ -2289,7 +2296,8 @@ logical_plan physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[array_agg(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]], 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] +03)----YieldStreamExec frequency=64 +04)------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? @@ -4298,7 +4306,8 @@ physical_plan 05)--------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 06)----------AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] +08)--------------YieldStreamExec frequency=64 +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 @@ -4393,7 +4402,8 @@ physical_plan 01)SortPreservingMergeExec: [name@0 DESC, time_chunks@1 DESC], fetch=5 02)--ProjectionExec: expr=[name@0 as name, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@1) as time_chunks] 03)----RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -04)------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] +04)------YieldStreamExec frequency=64 +05)--------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] statement ok drop table t1 diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 19763ab0083f..d1e243dbf2ea 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -682,7 +682,9 @@ query TT explain select * from t1 join t2 on false; ---- logical_plan EmptyRelation -physical_plan EmptyExec +physical_plan +01)YieldStreamExec frequency=64 +02)--EmptyExec # Make batch size smaller than table row number. to introduce parallelism to the plan. statement ok diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 86572421d76b..fb275d762d3a 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3615,12 +3615,15 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] 04)------ProjectionExec: expr=[1 as e, 3 as f] -05)--------PlaceholderRowExec -06)------UnionExec -07)--------ProjectionExec: expr=[1 as c, 2 as d] -08)----------PlaceholderRowExec -09)--------ProjectionExec: expr=[1 as c, 3 as d] -10)----------PlaceholderRowExec +05)--------YieldStreamExec frequency=64 +06)----------PlaceholderRowExec +07)------UnionExec +08)--------ProjectionExec: expr=[1 as c, 2 as d] +09)----------YieldStreamExec frequency=64 +10)------------PlaceholderRowExec +11)--------ProjectionExec: expr=[1 as c, 3 as d] +12)----------YieldStreamExec frequency=64 +13)------------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3658,12 +3661,15 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] 04)------ProjectionExec: expr=[1 as e, 3 as f] -05)--------PlaceholderRowExec -06)------UnionExec -07)--------ProjectionExec: expr=[1 as c, 2 as d] -08)----------PlaceholderRowExec -09)--------ProjectionExec: expr=[1 as c, 3 as d] -10)----------PlaceholderRowExec +05)--------YieldStreamExec frequency=64 +06)----------PlaceholderRowExec +07)------UnionExec +08)--------ProjectionExec: expr=[1 as c, 2 as d] +09)----------YieldStreamExec frequency=64 +10)------------PlaceholderRowExec +11)--------ProjectionExec: expr=[1 as c, 3 as d] +12)----------YieldStreamExec frequency=64 +13)------------PlaceholderRowExec query IIII rowsort SELECT * FROM ( diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 2f8944f462a1..ed1bfdf95bf4 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -313,7 +313,8 @@ logical_plan 04)------TableScan: t1 projection=[], fetch=14 physical_plan 01)ProjectionExec: expr=[0 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 11); @@ -332,7 +333,8 @@ logical_plan 04)------TableScan: t1 projection=[], fetch=11 physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -351,7 +353,8 @@ logical_plan 04)------TableScan: t1 projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--PlaceholderRowExec +02)--YieldStreamExec frequency=64 +03)----PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -552,7 +555,9 @@ explain SELECT * FROM data LIMIT 3; logical_plan 01)Limit: skip=0, fetch=3 02)--TableScan: data projection=[column1, column2], fetch=3 -physical_plan StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=3, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] +physical_plan +01)YieldStreamExec frequency=64 +02)--StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=3, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] # Do not remove limit with Sort when skip is used @@ -565,7 +570,8 @@ logical_plan 03)----TableScan: data projection=[column1, column2] physical_plan 01)GlobalLimitExec: skip=3, fetch=3 -02)--StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=6, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] +02)--YieldStreamExec frequency=64 +03)----StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=6, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] statement ok diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 3fc90a6459f2..bf73dd2aa02e 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -1307,9 +1307,11 @@ physical_plan 02)--ProjectionExec: expr=[CASE WHEN name@0 = name1 THEN 0 WHEN name@0 = name2 THEN 0.5 END as a] 03)----UnionExec 04)------ProjectionExec: expr=[name1 as name] -05)--------PlaceholderRowExec -06)------ProjectionExec: expr=[name2 as name] -07)--------PlaceholderRowExec +05)--------YieldStreamExec frequency=64 +06)----------PlaceholderRowExec +07)------ProjectionExec: expr=[name2 as name] +08)--------YieldStreamExec frequency=64 +09)----------PlaceholderRowExec query R SELECT diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index b4b31fa78a69..b33073ccc5d8 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -786,7 +786,9 @@ query TT explain select x from t where x IN (1,2,3) AND x IN (4,5); ---- logical_plan EmptyRelation -physical_plan EmptyExec +physical_plan +01)YieldStreamExec frequency=64 +02)--EmptyExec query TT explain select x from t where x NOT IN (1,2,3,4) OR x NOT IN (5,6,7,8); @@ -809,7 +811,9 @@ query TT explain select x from t where x NOT IN (1,2,3,4,5) AND x IN (1,2,3); ---- logical_plan EmptyRelation -physical_plan EmptyExec +physical_plan +01)YieldStreamExec frequency=64 +02)--EmptyExec statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 70666346e2ca..89c32f3f2546 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -125,7 +125,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 03)----FilterExec: c3@2 > 0 04)------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true +05)--------YieldStreamExec frequency=64 +06)----------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true # Start repratition on empty column test. # See https://github.com/apache/datafusion/issues/12057 diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 796570633f67..8738da67cf41 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -773,9 +773,11 @@ physical_plan 01)ProjectionExec: expr=[count(*)@0 as b, count(Int64(1))@1 as count(Int64(1))] 02)--NestedLoopJoinExec: join_type=Left 03)----ProjectionExec: expr=[4 as count(*)] -04)------PlaceholderRowExec -05)----ProjectionExec: expr=[4 as count(Int64(1))] -06)------PlaceholderRowExec +04)------YieldStreamExec frequency=64 +05)--------PlaceholderRowExec +06)----ProjectionExec: expr=[4 as count(Int64(1))] +07)------YieldStreamExec frequency=64 +08)--------PlaceholderRowExec statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index cff5c67b134e..77aed927a918 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -632,11 +632,14 @@ physical_plan 01)UnionExec 02)--ProjectionExec: expr=[Int64(1)@0 as a] 03)----AggregateExec: mode=SinglePartitioned, gby=[1 as Int64(1)], aggr=[], ordering_mode=Sorted -04)------PlaceholderRowExec -05)--ProjectionExec: expr=[2 as a] -06)----PlaceholderRowExec -07)--ProjectionExec: expr=[3 as a] -08)----PlaceholderRowExec +04)------YieldStreamExec frequency=64 +05)--------PlaceholderRowExec +06)--ProjectionExec: expr=[2 as a] +07)----YieldStreamExec frequency=64 +08)------PlaceholderRowExec +09)--ProjectionExec: expr=[3 as a] +10)----YieldStreamExec frequency=64 +11)------PlaceholderRowExec # test UNION ALL aliases correctly with aliased subquery query TT diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 8707177633bc..9390f59e81ec 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2846,7 +2846,8 @@ physical_plan 03)----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: false, 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] 04)------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: false, 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] 05)--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] -06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +06)----------YieldStreamExec frequency=64 +07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII SELECT @@ -2891,7 +2892,8 @@ physical_plan 03)----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: false, 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] 04)------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: false, 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] 05)--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] -06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +06)----------YieldStreamExec frequency=64 +07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII @@ -2995,7 +2997,8 @@ physical_plan 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 __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -10)------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +10)------------------YieldStreamExec frequency=64 +11)--------------------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, @@ -3131,7 +3134,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=4096, fetch=5 03)----FilterExec: row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 < 50 04)------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] -05)--------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] +05)--------YieldStreamExec frequency=64 +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] # Top level sort is pushed down through BoundedWindowAggExec as its SUM result does already satisfy the required # global order. The existing sort is for the second-term lexicographical ordering requirement, which is being @@ -3243,7 +3247,8 @@ physical_plan 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 __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -08)--------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +08)--------------YieldStreamExec frequency=64 +09)----------------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; @@ -3283,7 +3288,8 @@ physical_plan 14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -17)--------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +17)--------------------------------YieldStreamExec frequency=64 +18)----------------------------------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 @@ -3632,7 +3638,8 @@ physical_plan 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]] +07)------------YieldStreamExec frequency=64 +08)--------------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 From 187a945973b0d8c59d94d7f4d85c5677efd3f9d3 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 22:38:32 +0800 Subject: [PATCH 59/74] fix test --- .../core/tests/execution/infinite_cancel.rs | 61 +++++++++++-------- 1 file changed, 34 insertions(+), 27 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index d10948249357..e8884e7f947b 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -270,7 +270,7 @@ async fn test_infinite_sort_cancel( async fn test_infinite_interleave_cancel( #[values(false, true)] pretend_finite: bool, ) -> Result<(), Box> { - // 1) Build session, schema, and a sample batch. + // 1) Build a session and a schema with one i64 column. let session_ctx = SessionContext::new(); let schema = Arc::new(Schema::new(Fields::from(vec![Field::new( "value", @@ -283,17 +283,17 @@ async fn test_infinite_interleave_cancel( } let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(builder.finish())])?; - // 2) Create N infinite sources, each filtered by a different predicate. - // That way, the InterleaveExec will have multiple children. + // 2) Create multiple infinite sources, each filtered by a different threshold. + // This ensures InterleaveExec has many children. let mut infinite_children = vec![]; - // Use 32 distinct thresholds (each >0 and <8 192) to force 32 infinite inputs - let thresholds = (0..32).map(|i| 8_192 - 1 - (i * 256) as i64); + // Use 32 distinct thresholds (each > 0 and < 8192) for 32 infinite inputs. + let thresholds = (0..32).map(|i| 8191 - (i * 256) as i64); for thr in thresholds { - // 2a) One infinite exec: + // 2a) Construct an InfiniteExec for the sample batch. let inf = Arc::new(InfiniteExec::new(batch.clone(), pretend_finite)); - // 2b) Apply a FilterExec: “value > thr”. + // 2b) Apply a FilterExec with predicate "value > thr". let filter_expr = Arc::new(BinaryExpr::new( Arc::new(Column::new_with_schema("value", &schema)?), Gt, @@ -301,45 +301,50 @@ async fn test_infinite_interleave_cancel( )); let filtered = Arc::new(FilterExec::try_new(filter_expr, inf)?); - // 2c) Wrap in CoalesceBatchesExec so the upstream yields are batched. + // 2c) Wrap the filtered stream in CoalesceBatchesExec so it emits + // one 8192-row batch at a time. let coalesced = Arc::new(CoalesceBatchesExec::new(filtered, 8192)); - // 2d) Now repartition so that all children share identical Hash partitioning - // on “value” into 1 bucket. This is required for InterleaveExec::try_new. + // 2d) Repartition each coalesced stream by hashing on "value" into 1 partition. + // Required for InterleaveExec::try_new to succeed. let exprs = vec![Arc::new(Column::new_with_schema("value", &schema)?) as _]; let partitioning = Partitioning::Hash(exprs, 1); let hashed = Arc::new(RepartitionExec::try_new(coalesced, partitioning)?); - infinite_children.push(hashed as _); + infinite_children.push(hashed as Arc); } - // 3) Build an InterleaveExec over all N children. - // Since each child now has Partitioning::Hash([col "value"], 1), InterleaveExec::try_new succeeds. + // 3) Build an InterleaveExec over all infinite children. let interleave = Arc::new(InterleaveExec::try_new(infinite_children)?); - // 5) WrapLeaves will automatically insert YieldStreams beneath each “infinite” leaf. - // That way, each InfiniteExec (through the FilterExec/CoalesceBatchesExec/RepartitionExec chain) - // yields to the runtime periodically instead of spinning CPU. + // 4) Wrap the InterleaveExec in a FilterExec that always returns false, + // ensuring that no rows are ever emitted. + let always_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))); + let filtered_interleave = Arc::new(FilterExec::try_new(always_false, interleave)?); + + // 5) Coalesce the filtered interleave into 8192-row batches. + // This lets WrapLeaves insert YieldStreamExec at each batch boundary. + let coalesced_top = Arc::new(CoalesceBatchesExec::new(filtered_interleave, 8192)); + + // 6) Apply WrapLeaves to insert YieldStreamExec under every leaf. + // Each InfiniteExec → FilterExec → CoalesceBatchesExec chain will yield periodically. let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(interleave, &config)?; + let optimized = WrapLeaves::new().optimize(coalesced_top, &config)?; - // 6) Execute the stream. Because AggregateExec(mode=Single) only emits a final batch - // after all inputs finish—and those inputs are infinite—we expect no output - // within 1 second (timeout → None). + // 7) Execute the optimized plan with a 1-second timeout. + // Because the top-level FilterExec always discards rows and the inputs are infinite, + // no batch will be returned within 1 second, causing result to be None. let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; const TIMEOUT: u64 = 1; let result = select! { batch_opt = stream.next() => batch_opt, - _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => { - None - } + _ = tokio::time::sleep(tokio::time::Duration::from_secs(TIMEOUT)) => None, }; assert!( result.is_none(), - "Expected no output for aggregate over infinite interleave, but got some batch" + "Expected no output for infinite interleave aggregate, but got a batch" ); - Ok(()) } @@ -756,7 +761,8 @@ async fn test_infinite_hash_join_without_repartition_and_no_agg( // entirely, comment out the next line; however, not calling it is equivalent // because there is no aggregation so no wrapper is inserted. Here we simply do // not call WrapLeaves, ensuring the plan has neither aggregation nor repartition. - let optimized = join as Arc; + let config = ConfigOptions::new(); + let optimized = WrapLeaves::new().optimize(join, &config)?; // 4) Execute with a 1 second timeout let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; @@ -834,7 +840,8 @@ async fn test_infinite_sort_merge_join_without_repartition_and_no_agg( )?); // 3) Do not apply WrapLeaves (no aggregation, no repartition → no built-in yields). - let optimized = join as Arc; + let config = ConfigOptions::new(); + let optimized = WrapLeaves::new().optimize(join, &config)?; // 4) Execute with a 1-second timeout. Because both sides are infinite and never match, // the SortMergeJoin will never produce output within 1s. From 3cc4095243f408d1e08339e2e27faba6a6f9dbaa Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 23:21:17 +0800 Subject: [PATCH 60/74] add more built-in case --- .../physical-plan/src/placeholder_row.rs | 40 +++- datafusion/physical-plan/src/streaming.rs | 43 +++- datafusion/physical-plan/src/work_table.rs | 40 +++- .../sqllogictest/test_files/aggregate.slt | 61 ++---- datafusion/sqllogictest/test_files/array.slt | 6 +- .../test_files/count_star_rule.slt | 3 +- datafusion/sqllogictest/test_files/cte.slt | 89 ++++---- datafusion/sqllogictest/test_files/ddl.slt | 4 +- .../sqllogictest/test_files/explain.slt | 15 +- .../sqllogictest/test_files/explain_tree.slt | 201 ++++++------------ .../test_files/filter_without_sort_exec.slt | 18 +- .../sqllogictest/test_files/group_by.slt | 30 +-- datafusion/sqllogictest/test_files/joins.slt | 33 ++- datafusion/sqllogictest/test_files/limit.slt | 16 +- datafusion/sqllogictest/test_files/order.slt | 8 +- .../sqllogictest/test_files/repartition.slt | 3 +- .../sqllogictest/test_files/subquery.slt | 8 +- datafusion/sqllogictest/test_files/union.slt | 35 ++- datafusion/sqllogictest/test_files/window.slt | 57 ++--- 19 files changed, 326 insertions(+), 384 deletions(-) diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 46847b2413c0..2e21fdd569d7 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -31,6 +31,7 @@ use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; +use crate::yield_stream::YieldStream; use log::trace; /// Execution plan for empty relation with produce_one_row=true @@ -41,6 +42,8 @@ pub struct PlaceholderRowExec { /// Number of partitions partitions: usize, cache: PlanProperties, + /// Indicates whether to enable cooperative yielding mode. + cooperative: bool, } impl PlaceholderRowExec { @@ -52,6 +55,7 @@ impl PlaceholderRowExec { schema, partitions, cache, + cooperative: true, } } @@ -158,11 +162,25 @@ impl ExecutionPlan for PlaceholderRowExec { ); } - Ok(Box::pin(MemoryStream::try_new( - self.data()?, - Arc::clone(&self.schema), - None, - )?)) + let memory_stream = + MemoryStream::try_new(self.data()?, Arc::clone(&self.schema), None)?; + + let stream: SendableRecordBatchStream = Box::pin(memory_stream); + + if !self.cooperative { + return Ok(stream); + } + + let frequency = context + .session_config() + .options() + .optimizer + .yield_frequency_for_pipeline_break; + + let yielding = YieldStream::new(stream, frequency); + let yielding_stream: SendableRecordBatchStream = Box::pin(yielding); + + Ok(yielding_stream) } fn statistics(&self) -> Result { @@ -182,6 +200,18 @@ impl ExecutionPlan for PlaceholderRowExec { None, )) } + + /// Override: this operator *does* support cooperative yielding when `cooperative == true`. + fn yields_cooperatively(&self) -> bool { + self.cooperative + } + + /// If `cooperative == true`, return `Some(self.clone())` so the optimizer knows + /// we can replace a plain DataSourceExec with this same node (it already yields). + /// Otherwise, return None. + fn with_cooperative_yields(self: Arc) -> Option> { + self.cooperative.then_some(self) + } } #[cfg(test)] diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 6274995d04da..91929a5fc196 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -37,6 +37,7 @@ use datafusion_common::{internal_err, plan_err, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; +use crate::yield_stream::YieldStream; use async_trait::async_trait; use futures::stream::StreamExt; use log::debug; @@ -68,6 +69,8 @@ pub struct StreamingTableExec { limit: Option, cache: PlanProperties, metrics: ExecutionPlanMetricsSet, + /// Indicates whether to enable cooperative yielding mode. + cooperative: bool, } impl StreamingTableExec { @@ -112,6 +115,7 @@ impl StreamingTableExec { limit, cache, metrics: ExecutionPlanMetricsSet::new(), + cooperative: true, }) } @@ -262,7 +266,7 @@ impl ExecutionPlan for StreamingTableExec { partition: usize, ctx: Arc, ) -> Result { - let stream = self.partitions[partition].execute(ctx); + let stream = self.partitions[partition].execute(Arc::clone(&ctx)); let projected_stream = match self.projection.clone() { Some(projection) => Box::pin(RecordBatchStreamAdapter::new( Arc::clone(&self.projected_schema), @@ -272,6 +276,30 @@ impl ExecutionPlan for StreamingTableExec { )), None => stream, }; + + if self.cooperative { + let frequency = ctx + .session_config() + .options() + .optimizer + .yield_frequency_for_pipeline_break; + + let yielding_stream = YieldStream::new(projected_stream, frequency); + + return Ok(match self.limit { + None => Box::pin(yielding_stream), + Some(fetch) => { + let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + Box::pin(LimitStream::new( + Box::pin(yielding_stream), + 0, + Some(fetch), + baseline_metrics, + )) + } + }); + } + Ok(match self.limit { None => projected_stream, Some(fetch) => { @@ -347,8 +375,21 @@ impl ExecutionPlan for StreamingTableExec { limit, cache: self.cache.clone(), metrics: self.metrics.clone(), + cooperative: self.cooperative, })) } + + /// Override: this operator *does* support cooperative yielding when `cooperative == true`. + fn yields_cooperatively(&self) -> bool { + self.cooperative + } + + /// If `cooperative == true`, return `Some(self.clone())` so the optimizer knows + /// we can replace a plain DataSourceExec with this same node (it already yields). + /// Otherwise, return None. + fn with_cooperative_yields(self: Arc) -> Option> { + self.cooperative.then_some(self) + } } #[cfg(test)] diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index eea1b9958633..a4e086bc6cbb 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -28,6 +28,7 @@ use crate::{ }; use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; +use crate::yield_stream::YieldStream; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_datafusion_err, internal_err, Result}; @@ -106,6 +107,8 @@ pub struct WorkTableExec { metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Indicates whether to enable cooperative yielding mode. + cooperative: bool, } impl WorkTableExec { @@ -118,6 +121,7 @@ impl WorkTableExec { metrics: ExecutionPlanMetricsSet::new(), work_table: Arc::new(WorkTable::new()), cache, + cooperative: true, } } @@ -138,6 +142,7 @@ impl WorkTableExec { metrics: ExecutionPlanMetricsSet::new(), work_table, cache: self.cache.clone(), + cooperative: self.cooperative, } } @@ -205,7 +210,7 @@ impl ExecutionPlan for WorkTableExec { fn execute( &self, partition: usize, - _context: Arc, + context: Arc, ) -> Result { // WorkTable streams must be the plan base. if partition != 0 { @@ -214,10 +219,27 @@ impl ExecutionPlan for WorkTableExec { ); } let batch = self.work_table.take()?; - Ok(Box::pin( + + // 1. Get the “base” stream exactly as before, without yielding. + let stream = Box::pin( MemoryStream::try_new(batch.batches, Arc::clone(&self.schema), None)? .with_reservation(batch.reservation), - )) + ); + + // 2. If cooperative == false, return base_stream immediately. + if !self.cooperative { + return Ok(stream); + } + + let frequency = context + .session_config() + .options() + .optimizer + .yield_frequency_for_pipeline_break; + + // 3. If cooperative == true, wrap the stream into a YieldStream. + let yielding_stream = YieldStream::new(stream, frequency); + Ok(Box::pin(yielding_stream)) } fn metrics(&self) -> Option { @@ -231,6 +253,18 @@ impl ExecutionPlan for WorkTableExec { fn partition_statistics(&self, _partition: Option) -> Result { Ok(Statistics::new_unknown(&self.schema())) } + + /// Override: this operator *does* support cooperative yielding when `cooperative == true`. + fn yields_cooperatively(&self) -> bool { + self.cooperative + } + + /// If `cooperative == true`, return `Some(self.clone())` so the optimizer knows + /// we can replace a plain DataSourceExec with this same node (it already yields). + /// Otherwise, return None. + fn with_cooperative_yields(self: Arc) -> Option> { + self.cooperative.then_some(self) + } } #[cfg(test)] diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index eaa627a8747d..ed77435d6a85 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -380,20 +380,15 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[array_agg(DISTINCT a.foo), sum(DISTINCT Int64(1))], ordering_mode=Sorted 06)----------UnionExec 07)------------ProjectionExec: expr=[1 as id, 2 as foo] -08)--------------YieldStreamExec frequency=64 -09)----------------PlaceholderRowExec -10)------------ProjectionExec: expr=[1 as id, NULL as foo] -11)--------------YieldStreamExec frequency=64 -12)----------------PlaceholderRowExec -13)------------ProjectionExec: expr=[1 as id, NULL as foo] -14)--------------YieldStreamExec frequency=64 -15)----------------PlaceholderRowExec -16)------------ProjectionExec: expr=[1 as id, 3 as foo] -17)--------------YieldStreamExec frequency=64 -18)----------------PlaceholderRowExec -19)------------ProjectionExec: expr=[1 as id, 2 as foo] -20)--------------YieldStreamExec frequency=64 -21)----------------PlaceholderRowExec +08)--------------PlaceholderRowExec +09)------------ProjectionExec: expr=[1 as id, NULL as foo] +10)--------------PlaceholderRowExec +11)------------ProjectionExec: expr=[1 as id, NULL as foo] +12)--------------PlaceholderRowExec +13)------------ProjectionExec: expr=[1 as id, 3 as foo] +14)--------------PlaceholderRowExec +15)------------ProjectionExec: expr=[1 as id, 2 as foo] +16)--------------PlaceholderRowExec # FIX: custom absolute values @@ -6072,8 +6067,7 @@ logical_plan 02)--TableScan: empty projection=[col0] physical_plan 01)ProjectionExec: expr=[NULL as min(empty.col0)] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT EXPLAIN SELECT MAX(col0) FROM empty; @@ -6083,8 +6077,7 @@ logical_plan 02)--TableScan: empty projection=[col0] physical_plan 01)ProjectionExec: expr=[NULL as max(empty.col0)] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec statement ok DROP TABLE empty; @@ -6541,20 +6534,15 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[last_value(a.foo) ORDER BY [a.foo ASC NULLS LAST], sum(DISTINCT Int64(1))], ordering_mode=Sorted 06)----------UnionExec 07)------------ProjectionExec: expr=[1 as id, 2 as foo] -08)--------------YieldStreamExec frequency=64 -09)----------------PlaceholderRowExec -10)------------ProjectionExec: expr=[1 as id, 4 as foo] -11)--------------YieldStreamExec frequency=64 -12)----------------PlaceholderRowExec -13)------------ProjectionExec: expr=[1 as id, 5 as foo] -14)--------------YieldStreamExec frequency=64 -15)----------------PlaceholderRowExec -16)------------ProjectionExec: expr=[1 as id, 3 as foo] -17)--------------YieldStreamExec frequency=64 -18)----------------PlaceholderRowExec -19)------------ProjectionExec: expr=[1 as id, 2 as foo] -20)--------------YieldStreamExec frequency=64 -21)----------------PlaceholderRowExec +08)--------------PlaceholderRowExec +09)------------ProjectionExec: expr=[1 as id, 4 as foo] +10)--------------PlaceholderRowExec +11)------------ProjectionExec: expr=[1 as id, 5 as foo] +12)--------------PlaceholderRowExec +13)------------ProjectionExec: expr=[1 as id, 3 as foo] +14)--------------PlaceholderRowExec +15)------------ProjectionExec: expr=[1 as id, 2 as foo] +16)--------------PlaceholderRowExec # SortExec is removed if it is coming after one-row producing AggregateExec's having an empty group by expression query TT @@ -6684,8 +6672,7 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(Int64(1)), 2 as count()] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query II select count(1), count(*) from t; @@ -6701,8 +6688,7 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(Int64(1)), 2 as count(*)] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query II select count(), count(*) from t; @@ -6718,8 +6704,7 @@ logical_plan 03)----TableScan: t projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(), 2 as count(*)] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT explain select count(1) * count(2) from t; diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 5b23d94c4725..d89ba600d7a6 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -7196,8 +7196,7 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[true as array_has_all(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(1),Int64(3)))] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec # array containment operator with scalars #2 (arrow at) query BBBBBBB @@ -7220,8 +7219,7 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[true as array_has_all(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(1),Int64(3)))] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec ### Array casting tests diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index 4d0b92cd7d04..d38d3490fed4 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -37,8 +37,7 @@ logical_plan 04)------EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as count()] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT EXPLAIN SELECT t1.a, COUNT() FROM t1 GROUP BY t1.a; diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 4364688cac66..32320a06f4fb 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -36,8 +36,7 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec # cte_use_same_name_multiple_times statement error DataFusion error: Error during planning: WITH query name "a" specified more than once @@ -115,15 +114,13 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=nodes, is_distinct=false 02)--ProjectionExec: expr=[1 as id] -03)----YieldStreamExec frequency=64 -04)------PlaceholderRowExec -05)--CoalescePartitionsExec -06)----ProjectionExec: expr=[id@0 + 1 as id] -07)------CoalesceBatchesExec: target_batch_size=8192 -08)--------FilterExec: id@0 < 10 -09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------YieldStreamExec frequency=64 -11)--------------WorkTableExec: name=nodes +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 @@ -169,8 +166,7 @@ physical_plan 06)--------CoalesceBatchesExec: target_batch_size=2 07)----------FilterExec: time@0 < 10 08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------------YieldStreamExec frequency=64 -10)----------------WorkTableExec: name=balances +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 @@ -735,19 +731,16 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=recursive_cte, is_distinct=false 02)--ProjectionExec: expr=[1 as val] -03)----YieldStreamExec frequency=64 -04)------PlaceholderRowExec -05)--ProjectionExec: expr=[2 as val] -06)----CrossJoinExec -07)------CoalescePartitionsExec -08)--------CoalesceBatchesExec: target_batch_size=8182 -09)----------FilterExec: val@0 < 2 -10)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------YieldStreamExec frequency=64 -12)----------------WorkTableExec: name=recursive_cte -13)------ProjectionExec: expr=[2 as val] -14)--------YieldStreamExec frequency=64 -15)----------PlaceholderRowExec +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/datafusion/issues/9794 # Non-recursive term and recursive term have different types @@ -879,8 +872,7 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT explain with numbers(a,b,c) as (select 1,2,3) select * from numbers; @@ -891,8 +883,7 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT explain with numbers as (select 1 as a, 2 as b, 3 as c) select * from numbers; @@ -903,8 +894,7 @@ logical_plan 03)----EmptyRelation physical_plan 01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec statement count 0 create table person (id int, name string, primary key(id)) @@ -946,8 +936,7 @@ physical_plan 01)CrossJoinExec 02)--DataSourceExec: partitions=1, partition_sizes=[0] 03)--ProjectionExec: expr=[1 as Int64(1)] -04)----YieldStreamExec frequency=64 -05)------PlaceholderRowExec +04)----PlaceholderRowExec statement count 0 drop table j1; @@ -973,15 +962,13 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=numbers, is_distinct=false 02)--ProjectionExec: expr=[1 as n] -03)----YieldStreamExec frequency=64 -04)------PlaceholderRowExec -05)--CoalescePartitionsExec -06)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] -07)------CoalesceBatchesExec: target_batch_size=8182 -08)--------FilterExec: n@0 < 10 -09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------YieldStreamExec frequency=64 -11)--------------WorkTableExec: name=numbers +03)----PlaceholderRowExec +04)--CoalescePartitionsExec +05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] +06)------CoalesceBatchesExec: target_batch_size=8182 +07)--------FilterExec: n@0 < 10 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------WorkTableExec: name=numbers query TT explain WITH RECURSIVE numbers AS ( @@ -1001,15 +988,13 @@ logical_plan physical_plan 01)RecursiveQueryExec: name=numbers, is_distinct=false 02)--ProjectionExec: expr=[1 as n] -03)----YieldStreamExec frequency=64 -04)------PlaceholderRowExec -05)--CoalescePartitionsExec -06)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] -07)------CoalesceBatchesExec: target_batch_size=8182 -08)--------FilterExec: n@0 < 10 -09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------YieldStreamExec frequency=64 -11)--------------WorkTableExec: name=numbers +03)----PlaceholderRowExec +04)--CoalescePartitionsExec +05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] +06)------CoalesceBatchesExec: target_batch_size=8182 +07)--------FilterExec: n@0 < 10 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------WorkTableExec: name=numbers statement count 0 set datafusion.execution.enable_recursive_ctes = false; diff --git a/datafusion/sqllogictest/test_files/ddl.slt b/datafusion/sqllogictest/test_files/ddl.slt index 8ba9b0896c43..1e95e426f3e0 100644 --- a/datafusion/sqllogictest/test_files/ddl.slt +++ b/datafusion/sqllogictest/test_files/ddl.slt @@ -713,9 +713,7 @@ query TT explain select c1 from t; ---- logical_plan TableScan: t projection=[c1] -physical_plan -01)YieldStreamExec frequency=64 -02)--StreamingTableExec: partition_sizes=1, projection=[c1], infinite_source=true +physical_plan StreamingTableExec: partition_sizes=1, projection=[c1], infinite_source=true statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index b20c256a9801..8c9054d56c14 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -89,8 +89,7 @@ EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c ---- physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec statement ok set datafusion.explain.physical_plan_only = false @@ -386,8 +385,7 @@ logical_plan 02)--EmptyRelation physical_plan 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)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query TT explain select [[1, 2, 3], [4, 5, 6]]; @@ -397,8 +395,7 @@ logical_plan 02)--EmptyRelation physical_plan 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)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec # Explain Struct @@ -410,8 +407,7 @@ logical_plan 02)--EmptyRelation physical_plan 01)ProjectionExec: expr=[{c0:1,c1:2.3,c2:abc} as struct(Int64(1),Float64(2.3),Utf8("abc"))] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec statement ok @@ -434,8 +430,7 @@ physical_plan 01)NestedLoopJoinExec: join_type=LeftSemi 02)--DataSourceExec: partitions=1, partition_sizes=[0] 03)--ProjectionExec: expr=[] -04)----YieldStreamExec frequency=64 -05)------PlaceholderRowExec +04)----PlaceholderRowExec statement ok drop table t1; diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 77ddf4208927..15bf61576571 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -1151,17 +1151,11 @@ physical_plan 06)│ ASC NULLS LAST │ 07)└─────────────┬─────────────┘ 08)┌─────────────┴─────────────┐ -09)│ YieldStreamExec │ +09)│ StreamingTableExec │ 10)│ -------------------- │ -11)│ YieldStreamExec frequency:│ -12)│ 64 │ -13)└─────────────┬─────────────┘ -14)┌─────────────┴─────────────┐ -15)│ StreamingTableExec │ -16)│ -------------------- │ -17)│ infinite: true │ -18)│ limit: None │ -19)└───────────────────────────┘ +11)│ infinite: true │ +12)│ limit: None │ +13)└───────────────────────────┘ query TT EXPLAIN SELECT * @@ -1180,17 +1174,11 @@ physical_plan 08)│ limit: 50 │ 09)└─────────────┬─────────────┘ 10)┌─────────────┴─────────────┐ -11)│ YieldStreamExec │ +11)│ StreamingTableExec │ 12)│ -------------------- │ -13)│ YieldStreamExec frequency:│ -14)│ 64 │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ StreamingTableExec │ -18)│ -------------------- │ -19)│ infinite: true │ -20)│ limit: None │ -21)└───────────────────────────┘ +13)│ infinite: true │ +14)│ limit: None │ +15)└───────────────────────────┘ # Query with hash join. query TT @@ -1625,17 +1613,11 @@ physical_plan 25)│ RoundRobinBatch(4) │ 26)└─────────────┬─────────────┘ 27)┌─────────────┴─────────────┐ -28)│ YieldStreamExec │ +28)│ StreamingTableExec │ 29)│ -------------------- │ -30)│ YieldStreamExec frequency:│ -31)│ 64 │ -32)└─────────────┬─────────────┘ -33)┌─────────────┴─────────────┐ -34)│ StreamingTableExec │ -35)│ -------------------- │ -36)│ infinite: true │ -37)│ limit: None │ -38)└───────────────────────────┘ +30)│ infinite: true │ +31)│ limit: None │ +32)└───────────────────────────┘ # constant ticker, CAST(time AS DATE) = time, order by time @@ -1673,17 +1655,11 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ YieldStreamExec │ +29)│ StreamingTableExec │ 30)│ -------------------- │ -31)│ YieldStreamExec frequency:│ -32)│ 64 │ -33)└─────────────┬─────────────┘ -34)┌─────────────┴─────────────┐ -35)│ StreamingTableExec │ -36)│ -------------------- │ -37)│ infinite: true │ -38)│ limit: None │ -39)└───────────────────────────┘ +31)│ infinite: true │ +32)│ limit: None │ +33)└───────────────────────────┘ # same thing but order by date query TT @@ -1720,17 +1696,11 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ YieldStreamExec │ +29)│ StreamingTableExec │ 30)│ -------------------- │ -31)│ YieldStreamExec frequency:│ -32)│ 64 │ -33)└─────────────┬─────────────┘ -34)┌─────────────┴─────────────┐ -35)│ StreamingTableExec │ -36)│ -------------------- │ -37)│ infinite: true │ -38)│ limit: None │ -39)└───────────────────────────┘ +31)│ infinite: true │ +32)│ limit: None │ +33)└───────────────────────────┘ # same thing but order by ticker query TT @@ -1765,17 +1735,11 @@ physical_plan 24)│ RoundRobinBatch(4) │ 25)└─────────────┬─────────────┘ 26)┌─────────────┴─────────────┐ -27)│ YieldStreamExec │ +27)│ StreamingTableExec │ 28)│ -------------------- │ -29)│ YieldStreamExec frequency:│ -30)│ 64 │ -31)└─────────────┬─────────────┘ -32)┌─────────────┴─────────────┐ -33)│ StreamingTableExec │ -34)│ -------------------- │ -35)│ infinite: true │ -36)│ limit: None │ -37)└───────────────────────────┘ +29)│ infinite: true │ +30)│ limit: None │ +31)└───────────────────────────┘ # same thing but order by time, date @@ -1814,17 +1778,11 @@ physical_plan 27)│ RoundRobinBatch(4) │ 28)└─────────────┬─────────────┘ 29)┌─────────────┴─────────────┐ -30)│ YieldStreamExec │ +30)│ StreamingTableExec │ 31)│ -------------------- │ -32)│ YieldStreamExec frequency:│ -33)│ 64 │ -34)└─────────────┬─────────────┘ -35)┌─────────────┴─────────────┐ -36)│ StreamingTableExec │ -37)│ -------------------- │ -38)│ infinite: true │ -39)│ limit: None │ -40)└───────────────────────────┘ +32)│ infinite: true │ +33)│ limit: None │ +34)└───────────────────────────┘ @@ -1864,17 +1822,11 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ YieldStreamExec │ +29)│ StreamingTableExec │ 30)│ -------------------- │ -31)│ YieldStreamExec frequency:│ -32)│ 64 │ -33)└─────────────┬─────────────┘ -34)┌─────────────┴─────────────┐ -35)│ StreamingTableExec │ -36)│ -------------------- │ -37)│ infinite: true │ -38)│ limit: None │ -39)└───────────────────────────┘ +31)│ infinite: true │ +32)│ limit: None │ +33)└───────────────────────────┘ @@ -1899,42 +1851,35 @@ physical_plan 07)│ id: 1 ││ │ 08)└─────────────┬─────────────┘└─────────────┬─────────────┘ 09)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -10)│ YieldStreamExec ││ ProjectionExec │ -11)│ -------------------- ││ -------------------- │ -12)│ YieldStreamExec frequency:││ id: id + 1 │ -13)│ 64 ││ │ -14)└─────────────┬─────────────┘└─────────────┬─────────────┘ -15)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -16)│ PlaceholderRowExec ││ CoalesceBatchesExec │ -17)│ ││ -------------------- │ -18)│ ││ target_batch_size: │ -19)│ ││ 8192 │ -20)└───────────────────────────┘└─────────────┬─────────────┘ -21)-----------------------------┌─────────────┴─────────────┐ -22)-----------------------------│ FilterExec │ -23)-----------------------------│ -------------------- │ -24)-----------------------------│ predicate: id < 10 │ -25)-----------------------------└─────────────┬─────────────┘ -26)-----------------------------┌─────────────┴─────────────┐ -27)-----------------------------│ RepartitionExec │ -28)-----------------------------│ -------------------- │ -29)-----------------------------│ partition_count(in->out): │ -30)-----------------------------│ 1 -> 4 │ -31)-----------------------------│ │ -32)-----------------------------│ partitioning_scheme: │ -33)-----------------------------│ RoundRobinBatch(4) │ -34)-----------------------------└─────────────┬─────────────┘ -35)-----------------------------┌─────────────┴─────────────┐ -36)-----------------------------│ YieldStreamExec │ -37)-----------------------------│ -------------------- │ -38)-----------------------------│ YieldStreamExec frequency:│ -39)-----------------------------│ 64 │ -40)-----------------------------└─────────────┬─────────────┘ -41)-----------------------------┌─────────────┴─────────────┐ -42)-----------------------------│ WorkTableExec │ -43)-----------------------------│ -------------------- │ -44)-----------------------------│ name: nodes │ -45)-----------------------------└───────────────────────────┘ +10)│ PlaceholderRowExec ││ ProjectionExec │ +11)│ ││ -------------------- │ +12)│ ││ id: id + 1 │ +13)└───────────────────────────┘└─────────────┬─────────────┘ +14)-----------------------------┌─────────────┴─────────────┐ +15)-----------------------------│ CoalesceBatchesExec │ +16)-----------------------------│ -------------------- │ +17)-----------------------------│ target_batch_size: │ +18)-----------------------------│ 8192 │ +19)-----------------------------└─────────────┬─────────────┘ +20)-----------------------------┌─────────────┴─────────────┐ +21)-----------------------------│ FilterExec │ +22)-----------------------------│ -------------------- │ +23)-----------------------------│ predicate: id < 10 │ +24)-----------------------------└─────────────┬─────────────┘ +25)-----------------------------┌─────────────┴─────────────┐ +26)-----------------------------│ RepartitionExec │ +27)-----------------------------│ -------------------- │ +28)-----------------------------│ partition_count(in->out): │ +29)-----------------------------│ 1 -> 4 │ +30)-----------------------------│ │ +31)-----------------------------│ partitioning_scheme: │ +32)-----------------------------│ RoundRobinBatch(4) │ +33)-----------------------------└─────────────┬─────────────┘ +34)-----------------------------┌─────────────┴─────────────┐ +35)-----------------------------│ WorkTableExec │ +36)-----------------------------│ -------------------- │ +37)-----------------------------│ name: nodes │ +38)-----------------------------└───────────────────────────┘ query TT explain COPY (VALUES (1, 'foo', 1, '2023-01-01'), (2, 'bar', 2, '2023-01-02'), (3, 'baz', 3, '2023-01-03')) @@ -2124,17 +2069,11 @@ physical_plan 26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ YieldStreamExec │ +29)│ StreamingTableExec │ 30)│ -------------------- │ -31)│ YieldStreamExec frequency:│ -32)│ 64 │ -33)└─────────────┬─────────────┘ -34)┌─────────────┴─────────────┐ -35)│ StreamingTableExec │ -36)│ -------------------- │ -37)│ infinite: true │ -38)│ limit: None │ -39)└───────────────────────────┘ +31)│ infinite: true │ +32)│ limit: None │ +33)└───────────────────────────┘ # Test explain tree for PlaceholderRowExec query TT @@ -2147,11 +2086,5 @@ physical_plan 04)│ count(*): 2 │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ YieldStreamExec │ -08)│ -------------------- │ -09)│ YieldStreamExec frequency:│ -10)│ 64 │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ PlaceholderRowExec │ -14)└───────────────────────────┘ +07)│ PlaceholderRowExec │ +08)└───────────────────────────┘ diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index 6e74466bd10c..a09d8ce26ddf 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -41,8 +41,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------YieldStreamExec frequency=64 -06)----------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] +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 @@ -59,8 +58,7 @@ physical_plan 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)--------YieldStreamExec frequency=64 -06)----------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] +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 @@ -77,8 +75,7 @@ physical_plan 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)--------YieldStreamExec frequency=64 -06)----------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] +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 @@ -95,8 +92,7 @@ physical_plan 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)--------YieldStreamExec frequency=64 -06)----------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] +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 @@ -113,8 +109,7 @@ physical_plan 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)--------YieldStreamExec frequency=64 -06)----------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] +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 @@ -155,5 +150,4 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: date@0 = 2006-01-02 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------YieldStreamExec frequency=64 -06)----------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] +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 6b030586afdb..9e67018ecd0b 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2112,8 +2112,7 @@ logical_plan physical_plan 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)----YieldStreamExec frequency=64 -04)------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] +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 @@ -2144,8 +2143,7 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=PartiallySorted([1]) -03)----YieldStreamExec frequency=64 -04)------StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] +03)----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] query III SELECT a, d, @@ -2178,8 +2176,7 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted -03)----YieldStreamExec frequency=64 -04)------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] +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 @@ -2205,8 +2202,7 @@ logical_plan physical_plan 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) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted -03)----YieldStreamExec frequency=64 -04)------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] +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 @@ -2233,8 +2229,7 @@ logical_plan physical_plan 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)----YieldStreamExec frequency=64 -04)------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] +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 c) as last_c @@ -2256,8 +2251,7 @@ logical_plan 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan 01)PartialSortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], common_prefix_length=[2] -02)--YieldStreamExec frequency=64 -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] +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 * @@ -2270,8 +2264,7 @@ logical_plan 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan 01)PartialSortExec: TopK(fetch=50), expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], common_prefix_length=[2] -02)--YieldStreamExec frequency=64 -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] +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 * @@ -2296,8 +2289,7 @@ logical_plan physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[array_agg(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]], 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)----YieldStreamExec frequency=64 -04)------StreamingTableExec: partition_sizes=1, projection=[a, b, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST] +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? @@ -4306,8 +4298,7 @@ physical_plan 05)--------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 06)----------AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------YieldStreamExec frequency=64 -09)----------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] +08)--------------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 @@ -4402,8 +4393,7 @@ physical_plan 01)SortPreservingMergeExec: [name@0 DESC, time_chunks@1 DESC], fetch=5 02)--ProjectionExec: expr=[name@0 as name, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@1) as time_chunks] 03)----RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -04)------YieldStreamExec frequency=64 -05)--------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] +04)------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] statement ok drop table t1 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index fb275d762d3a..ccecb9494331 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3615,15 +3615,12 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] 04)------ProjectionExec: expr=[1 as e, 3 as f] -05)--------YieldStreamExec frequency=64 -06)----------PlaceholderRowExec -07)------UnionExec -08)--------ProjectionExec: expr=[1 as c, 2 as d] -09)----------YieldStreamExec frequency=64 -10)------------PlaceholderRowExec -11)--------ProjectionExec: expr=[1 as c, 3 as d] -12)----------YieldStreamExec frequency=64 -13)------------PlaceholderRowExec +05)--------PlaceholderRowExec +06)------UnionExec +07)--------ProjectionExec: expr=[1 as c, 2 as d] +08)----------PlaceholderRowExec +09)--------ProjectionExec: expr=[1 as c, 3 as d] +10)----------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3661,15 +3658,12 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] 04)------ProjectionExec: expr=[1 as e, 3 as f] -05)--------YieldStreamExec frequency=64 -06)----------PlaceholderRowExec -07)------UnionExec -08)--------ProjectionExec: expr=[1 as c, 2 as d] -09)----------YieldStreamExec frequency=64 -10)------------PlaceholderRowExec -11)--------ProjectionExec: expr=[1 as c, 3 as d] -12)----------YieldStreamExec frequency=64 -13)------------PlaceholderRowExec +05)--------PlaceholderRowExec +06)------UnionExec +07)--------ProjectionExec: expr=[1 as c, 2 as d] +08)----------PlaceholderRowExec +09)--------ProjectionExec: expr=[1 as c, 3 as d] +10)----------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -4708,8 +4702,7 @@ physical_plan 01)CrossJoinExec 02)--DataSourceExec: partitions=1, partition_sizes=[0] 03)--ProjectionExec: expr=[1 as Int64(1)] -04)----YieldStreamExec frequency=64 -05)------PlaceholderRowExec +04)----PlaceholderRowExec statement count 0 drop table j1; diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index ed1bfdf95bf4..2f8944f462a1 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -313,8 +313,7 @@ logical_plan 04)------TableScan: t1 projection=[], fetch=14 physical_plan 01)ProjectionExec: expr=[0 as count(*)] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 11); @@ -333,8 +332,7 @@ logical_plan 04)------TableScan: t1 projection=[], fetch=11 physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -353,8 +351,7 @@ logical_plan 04)------TableScan: t1 projection=[] physical_plan 01)ProjectionExec: expr=[2 as count(*)] -02)--YieldStreamExec frequency=64 -03)----PlaceholderRowExec +02)--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -555,9 +552,7 @@ explain SELECT * FROM data LIMIT 3; logical_plan 01)Limit: skip=0, fetch=3 02)--TableScan: data projection=[column1, column2], fetch=3 -physical_plan -01)YieldStreamExec frequency=64 -02)--StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=3, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] +physical_plan StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=3, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] # Do not remove limit with Sort when skip is used @@ -570,8 +565,7 @@ logical_plan 03)----TableScan: data projection=[column1, column2] physical_plan 01)GlobalLimitExec: skip=3, fetch=3 -02)--YieldStreamExec frequency=64 -03)----StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=6, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] +02)--StreamingTableExec: partition_sizes=1, projection=[column1, column2], infinite_source=true, fetch=6, output_ordering=[column1@0 ASC NULLS LAST, column2@1 ASC NULLS LAST] statement ok diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index bf73dd2aa02e..3fc90a6459f2 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -1307,11 +1307,9 @@ physical_plan 02)--ProjectionExec: expr=[CASE WHEN name@0 = name1 THEN 0 WHEN name@0 = name2 THEN 0.5 END as a] 03)----UnionExec 04)------ProjectionExec: expr=[name1 as name] -05)--------YieldStreamExec frequency=64 -06)----------PlaceholderRowExec -07)------ProjectionExec: expr=[name2 as name] -08)--------YieldStreamExec frequency=64 -09)----------PlaceholderRowExec +05)--------PlaceholderRowExec +06)------ProjectionExec: expr=[name2 as name] +07)--------PlaceholderRowExec query R SELECT diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 89c32f3f2546..70666346e2ca 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -125,8 +125,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 03)----FilterExec: c3@2 > 0 04)------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -05)--------YieldStreamExec frequency=64 -06)----------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true +05)--------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true # Start repratition on empty column test. # See https://github.com/apache/datafusion/issues/12057 diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 8738da67cf41..796570633f67 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -773,11 +773,9 @@ physical_plan 01)ProjectionExec: expr=[count(*)@0 as b, count(Int64(1))@1 as count(Int64(1))] 02)--NestedLoopJoinExec: join_type=Left 03)----ProjectionExec: expr=[4 as count(*)] -04)------YieldStreamExec frequency=64 -05)--------PlaceholderRowExec -06)----ProjectionExec: expr=[4 as count(Int64(1))] -07)------YieldStreamExec frequency=64 -08)--------PlaceholderRowExec +04)------PlaceholderRowExec +05)----ProjectionExec: expr=[4 as count(Int64(1))] +06)------PlaceholderRowExec statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 77aed927a918..d549f555f9d8 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -520,13 +520,11 @@ physical_plan 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true 16)----ProjectionExec: expr=[1 as cnt] -17)------YieldStreamExec frequency=64 -18)--------PlaceholderRowExec -19)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] -20)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) 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 }], mode=[Sorted] -21)--------ProjectionExec: expr=[1 as c1] -22)----------YieldStreamExec frequency=64 -23)------------PlaceholderRowExec +17)------PlaceholderRowExec +18)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] +19)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) 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 }], mode=[Sorted] +20)--------ProjectionExec: expr=[1 as c1] +21)----------PlaceholderRowExec ######## @@ -632,14 +630,11 @@ physical_plan 01)UnionExec 02)--ProjectionExec: expr=[Int64(1)@0 as a] 03)----AggregateExec: mode=SinglePartitioned, gby=[1 as Int64(1)], aggr=[], ordering_mode=Sorted -04)------YieldStreamExec frequency=64 -05)--------PlaceholderRowExec -06)--ProjectionExec: expr=[2 as a] -07)----YieldStreamExec frequency=64 -08)------PlaceholderRowExec -09)--ProjectionExec: expr=[3 as a] -10)----YieldStreamExec frequency=64 -11)------PlaceholderRowExec +04)------PlaceholderRowExec +05)--ProjectionExec: expr=[2 as a] +06)----PlaceholderRowExec +07)--ProjectionExec: expr=[3 as a] +08)----PlaceholderRowExec # test UNION ALL aliases correctly with aliased subquery query TT @@ -664,12 +659,10 @@ physical_plan 02)--ProjectionExec: expr=[count(Int64(1))@1 as count, n@0 as n] 03)----AggregateExec: mode=SinglePartitioned, gby=[n@0 as n], aggr=[count(Int64(1))], ordering_mode=Sorted 04)------ProjectionExec: expr=[5 as n] -05)--------YieldStreamExec frequency=64 -06)----------PlaceholderRowExec -07)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] -08)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] -09)------YieldStreamExec frequency=64 -10)--------PlaceholderRowExec +05)--------PlaceholderRowExec +06)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] +07)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] +08)------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/11409 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 9390f59e81ec..c86921012f9b 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -280,17 +280,13 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[max(d.a)], ordering_mode=Sorted 08)--------------UnionExec 09)----------------ProjectionExec: expr=[1 as a, aa as b] -10)------------------YieldStreamExec frequency=64 -11)--------------------PlaceholderRowExec -12)----------------ProjectionExec: expr=[3 as a, aa as b] -13)------------------YieldStreamExec frequency=64 -14)--------------------PlaceholderRowExec -15)----------------ProjectionExec: expr=[5 as a, bb as b] -16)------------------YieldStreamExec frequency=64 -17)--------------------PlaceholderRowExec -18)----------------ProjectionExec: expr=[7 as a, bb as b] -19)------------------YieldStreamExec frequency=64 -20)--------------------PlaceholderRowExec +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 @@ -370,17 +366,13 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4 09)----------------UnionExec 10)------------------ProjectionExec: expr=[1 as a, aa as b] -11)--------------------YieldStreamExec frequency=64 -12)----------------------PlaceholderRowExec -13)------------------ProjectionExec: expr=[3 as a, aa as b] -14)--------------------YieldStreamExec frequency=64 -15)----------------------PlaceholderRowExec -16)------------------ProjectionExec: expr=[5 as a, bb as b] -17)--------------------YieldStreamExec frequency=64 -18)----------------------PlaceholderRowExec -19)------------------ProjectionExec: expr=[7 as a, bb as b] -20)--------------------YieldStreamExec frequency=64 -21)----------------------PlaceholderRowExec +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 @@ -2846,8 +2838,7 @@ physical_plan 03)----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: false, 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] 04)------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: false, 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] 05)--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] -06)----------YieldStreamExec frequency=64 -07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII SELECT @@ -2892,8 +2883,7 @@ physical_plan 03)----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: false, 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] 04)------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: false, 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] 05)--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] -06)----------YieldStreamExec frequency=64 -07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII @@ -2997,8 +2987,7 @@ physical_plan 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 __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -10)------------------YieldStreamExec frequency=64 -11)--------------------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] +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, @@ -3134,8 +3123,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=4096, fetch=5 03)----FilterExec: row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 < 50 04)------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] -05)--------YieldStreamExec frequency=64 -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] +05)--------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] # Top level sort is pushed down through BoundedWindowAggExec as its SUM result does already satisfy the required # global order. The existing sort is for the second-term lexicographical ordering requirement, which is being @@ -3247,8 +3235,7 @@ physical_plan 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 __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -08)--------------YieldStreamExec frequency=64 -09)----------------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] +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; @@ -3288,8 +3275,7 @@ physical_plan 14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -17)--------------------------------YieldStreamExec frequency=64 -18)----------------------------------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] +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 @@ -3638,8 +3624,7 @@ physical_plan 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)------------YieldStreamExec frequency=64 -08)--------------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]] +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 From 4614048282a7bd58a2068281e13e28d0db004e6d Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 23:26:35 +0800 Subject: [PATCH 61/74] fix --- datafusion/core/src/physical_planner.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 61d1fee79472..77410f4b08ce 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -3049,8 +3049,10 @@ mod tests { digraph { 1[shape=box label="ProjectionExec: expr=[id@0 + 2 as employee.id + Int32(2)]", tooltip=""] - 2[shape=box label="EmptyExec", tooltip=""] + 2[shape=box label="YieldStreamExec frequency=64", tooltip=""] 1 -> 2 [arrowhead=none, arrowtail=normal, dir=back] + 3[shape=box label="EmptyExec", tooltip=""] + 2 -> 3 [arrowhead=none, arrowtail=normal, dir=back] } // End DataFusion GraphViz Plan "#; From 80475346c53d482989afc6b0c1a93e1ddaa6bee3 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 6 Jun 2025 23:48:38 +0800 Subject: [PATCH 62/74] fix user defined exec --- .../core/tests/user_defined/insert_operation.rs | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/datafusion/core/tests/user_defined/insert_operation.rs b/datafusion/core/tests/user_defined/insert_operation.rs index 12f700ce572b..4db05ea2c729 100644 --- a/datafusion/core/tests/user_defined/insert_operation.rs +++ b/datafusion/core/tests/user_defined/insert_operation.rs @@ -26,6 +26,7 @@ use datafusion::{ use datafusion_catalog::{Session, TableProvider}; use datafusion_expr::{dml::InsertOp, Expr, TableType}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::{ execution_plan::{Boundedness, EmissionType}, DisplayAs, ExecutionPlan, PlanProperties, @@ -55,7 +56,15 @@ async fn insert_operation_is_passed_correctly_to_table_provider() { async fn assert_insert_op(ctx: &SessionContext, sql: &str, insert_op: InsertOp) { let df = ctx.sql(sql).await.unwrap(); let plan = df.create_physical_plan().await.unwrap(); - let exec = plan.as_any().downcast_ref::().unwrap(); + let maybe_yield = plan.as_any().downcast_ref::(); + + let target: &dyn ExecutionPlan = if let Some(yield_exec) = maybe_yield { + // `yield_exec.input()` is Arc → call .as_ref() to get &dyn ExecutionPlan + yield_exec.input().as_ref() + } else { + plan.as_ref() // plan is an Arc → as_ref() gives &dyn ExecutionPlan + }; + let exec = target.as_any().downcast_ref::().unwrap(); assert_eq!(exec.op, insert_op); } From 7f4c93e8474b03bcc99409efd2776d089acfb93f Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 6 Jun 2025 20:33:10 +0300 Subject: [PATCH 63/74] Reduce test diff --- datafusion/core/src/physical_planner.rs | 4 +--- .../tests/user_defined/insert_operation.rs | 19 +++++++++---------- datafusion/datasource/src/sink.rs | 8 ++++++++ datafusion/physical-plan/src/empty.rs | 8 ++++++++ .../sqllogictest/test_files/join.slt.part | 4 +--- .../sqllogictest/test_files/predicates.slt | 8 ++------ 6 files changed, 29 insertions(+), 22 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 77410f4b08ce..61d1fee79472 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -3049,10 +3049,8 @@ mod tests { digraph { 1[shape=box label="ProjectionExec: expr=[id@0 + 2 as employee.id + Int32(2)]", tooltip=""] - 2[shape=box label="YieldStreamExec frequency=64", tooltip=""] + 2[shape=box label="EmptyExec", tooltip=""] 1 -> 2 [arrowhead=none, arrowtail=normal, dir=back] - 3[shape=box label="EmptyExec", tooltip=""] - 2 -> 3 [arrowhead=none, arrowtail=normal, dir=back] } // End DataFusion GraphViz Plan "#; diff --git a/datafusion/core/tests/user_defined/insert_operation.rs b/datafusion/core/tests/user_defined/insert_operation.rs index 4db05ea2c729..94523fb372ea 100644 --- a/datafusion/core/tests/user_defined/insert_operation.rs +++ b/datafusion/core/tests/user_defined/insert_operation.rs @@ -26,7 +26,6 @@ use datafusion::{ use datafusion_catalog::{Session, TableProvider}; use datafusion_expr::{dml::InsertOp, Expr, TableType}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; -use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::{ execution_plan::{Boundedness, EmissionType}, DisplayAs, ExecutionPlan, PlanProperties, @@ -56,15 +55,7 @@ async fn insert_operation_is_passed_correctly_to_table_provider() { async fn assert_insert_op(ctx: &SessionContext, sql: &str, insert_op: InsertOp) { let df = ctx.sql(sql).await.unwrap(); let plan = df.create_physical_plan().await.unwrap(); - let maybe_yield = plan.as_any().downcast_ref::(); - - let target: &dyn ExecutionPlan = if let Some(yield_exec) = maybe_yield { - // `yield_exec.input()` is Arc → call .as_ref() to get &dyn ExecutionPlan - yield_exec.input().as_ref() - } else { - plan.as_ref() // plan is an Arc → as_ref() gives &dyn ExecutionPlan - }; - let exec = target.as_any().downcast_ref::().unwrap(); + let exec = plan.as_any().downcast_ref::().unwrap(); assert_eq!(exec.op, insert_op); } @@ -188,6 +179,14 @@ impl ExecutionPlan for TestInsertExec { ) -> Result { unimplemented!("TestInsertExec is a stub for testing.") } + + fn yields_cooperatively(&self) -> bool { + true + } + + fn with_cooperative_yields(self: Arc) -> Option> { + Some(self) + } } fn make_count_schema() -> SchemaRef { diff --git a/datafusion/datasource/src/sink.rs b/datafusion/datasource/src/sink.rs index 0552370d8ed0..6a2f36f65854 100644 --- a/datafusion/datasource/src/sink.rs +++ b/datafusion/datasource/src/sink.rs @@ -248,6 +248,14 @@ impl ExecutionPlan for DataSinkExec { fn metrics(&self) -> Option { self.sink.metrics() } + + fn yields_cooperatively(&self) -> bool { + true + } + + fn with_cooperative_yields(self: Arc) -> Option> { + Some(self) + } } /// Create a output record batch with a count diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 36634fbe6d7e..7507e05738da 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -173,6 +173,14 @@ impl ExecutionPlan for EmptyExec { None, )) } + + fn yields_cooperatively(&self) -> bool { + true + } + + fn with_cooperative_yields(self: Arc) -> Option> { + Some(self) + } } #[cfg(test)] diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index d1e243dbf2ea..19763ab0083f 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -682,9 +682,7 @@ query TT explain select * from t1 join t2 on false; ---- logical_plan EmptyRelation -physical_plan -01)YieldStreamExec frequency=64 -02)--EmptyExec +physical_plan EmptyExec # Make batch size smaller than table row number. to introduce parallelism to the plan. statement ok diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index b33073ccc5d8..b4b31fa78a69 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -786,9 +786,7 @@ query TT explain select x from t where x IN (1,2,3) AND x IN (4,5); ---- logical_plan EmptyRelation -physical_plan -01)YieldStreamExec frequency=64 -02)--EmptyExec +physical_plan EmptyExec query TT explain select x from t where x NOT IN (1,2,3,4) OR x NOT IN (5,6,7,8); @@ -811,9 +809,7 @@ query TT explain select x from t where x NOT IN (1,2,3,4,5) AND x IN (1,2,3); ---- logical_plan EmptyRelation -physical_plan -01)YieldStreamExec frequency=64 -02)--EmptyExec +physical_plan EmptyExec statement ok drop table t; From ca7f061f3a117c4c8510725a8a2e17ff927ba952 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 6 Jun 2025 20:52:09 +0300 Subject: [PATCH 64/74] Format imports --- datafusion/physical-plan/src/streaming.rs | 2 +- datafusion/physical-plan/src/work_table.rs | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 91929a5fc196..7495c03029b3 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -30,6 +30,7 @@ use crate::projection::{ all_alias_free_columns, new_projections_for_columns, update_expr, ProjectionExec, }; use crate::stream::RecordBatchStreamAdapter; +use crate::yield_stream::YieldStream; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; use arrow::datatypes::{Schema, SchemaRef}; @@ -37,7 +38,6 @@ use datafusion_common::{internal_err, plan_err, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; -use crate::yield_stream::YieldStream; use async_trait::async_trait; use futures::stream::StreamExt; use log::debug; diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index a4e086bc6cbb..e60fb58d94a8 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -22,13 +22,13 @@ use std::sync::{Arc, Mutex}; use crate::execution_plan::{Boundedness, EmissionType}; use crate::memory::MemoryStream; +use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::yield_stream::YieldStream; use crate::{ - metrics::{ExecutionPlanMetricsSet, MetricsSet}, + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SendableRecordBatchStream, Statistics, }; -use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; -use crate::yield_stream::YieldStream; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_datafusion_err, internal_err, Result}; From 5806bdb1d68d804ef453fb2b603efeada3c94b9a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 6 Jun 2025 22:13:10 +0300 Subject: [PATCH 65/74] Update documentation --- datafusion/common/src/config.rs | 15 +++++++-------- docs/source/user-guide/configs.md | 2 +- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 5645b89449e8..30d1f1c7a545 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -723,14 +723,13 @@ config_namespace! { /// Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. pub expand_views_at_output: bool, default = false - /// Yield frequency in batches, it represents how many batches to process before yielding - /// to the Tokio scheduler. The default value is 64, which means that after processing - /// 64 batches, the execution will yield control back to the Tokio scheduler. - /// This value should be greater than 0 or equal to 0; - /// When it is greater than 0, `enable` the optimizer will insert a Yield operator at the leaf nodes of any pipeline - /// that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to - /// other tasks while waiting. - /// If you want to `disable` the Yield operator, you can set this value to 0. + /// When DataFusion detects that a plan might not be promply cancellable + /// due to the presence of tight-looping operators, it will attempt to + /// mitigate this by inserting explicit yielding (in as few places as + /// possible to avoid performance degradation). This value represents the + /// yielding frequency (in batches) at such explicit yielding points. + /// The default value is 64. If set to 0, no DataFusion will not perform + /// any explicit yielding. pub yield_frequency_for_pipeline_break: usize, default = 64 } } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 4b05d4827b2d..abd9454a1e4a 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -119,7 +119,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | | datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | | datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | -| datafusion.optimizer.yield_frequency_for_pipeline_break | 64 | Yield frequency in batches, it represents how many batches to process before yielding to the Tokio scheduler. The default value is 64, which means that after processing 64 batches, the execution will yield control back to the Tokio scheduler. This value should be greater than 0 or equal to 0; When it is greater than 0, `enable` the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. If you want to `disable` the Yield operator, you can set this value to 0. | +| datafusion.optimizer.yield_frequency_for_pipeline_break | 64 | When DataFusion detects that a plan might not be promply cancellable due to the presence of tight-looping operators, it will attempt to mitigate this by inserting explicit yielding (in as few places as possible to avoid performance degradation). This value represents the yielding frequency (in batches) at such explicit yielding points. The default value is 64. If set to 0, no DataFusion will not perform any explicit yielding. | | datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | | datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | | datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | From b45d3dc4ae6b46549d83cc063ed8b7307bf81ecf Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 6 Jun 2025 22:15:54 +0300 Subject: [PATCH 66/74] Update information schema per docs --- datafusion/sqllogictest/test_files/information_schema.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index a484eca6717c..e45b69e549de 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -414,7 +414,7 @@ datafusion.optimizer.repartition_sorts true Should DataFusion execute sorts in a datafusion.optimizer.repartition_windows true Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level datafusion.optimizer.skip_failed_rules false When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail datafusion.optimizer.top_down_join_key_reordering true When set to true, the physical plan optimizer will run a top down process to reorder the join keys -datafusion.optimizer.yield_frequency_for_pipeline_break 64 Yield frequency in batches, it represents how many batches to process before yielding to the Tokio scheduler. The default value is 64, which means that after processing 64 batches, the execution will yield control back to the Tokio scheduler. This value should be greater than 0 or equal to 0; When it is greater than 0, `enable` the optimizer will insert a Yield operator at the leaf nodes of any pipeline that contains a pipeline-breaking operator, allowing the Tokio scheduler to switch to other tasks while waiting. If you want to `disable` the Yield operator, you can set this value to 0. +datafusion.optimizer.yield_frequency_for_pipeline_break 64 When DataFusion detects that a plan might not be promply cancellable due to the presence of tight-looping operators, it will attempt to mitigate this by inserting explicit yielding (in as few places as possible to avoid performance degradation). This value represents the yielding frequency (in batches) at such explicit yielding points. The default value is 64. If set to 0, no DataFusion will not perform any explicit yielding. datafusion.sql_parser.collect_spans false When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. datafusion.sql_parser.dialect generic Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. datafusion.sql_parser.enable_ident_normalization true When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) From a21c6ce69454a5fdd08bf2b45aefa98e80f55337 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 6 Jun 2025 23:04:38 +0300 Subject: [PATCH 67/74] Only retain the with_cooperative_yields API for now --- .../tests/user_defined/insert_operation.rs | 4 - datafusion/datasource/src/sink.rs | 4 - datafusion/datasource/src/source.rs | 8 -- .../src/wrap_leaves_cancellation.rs | 87 +++++++------------ datafusion/physical-plan/src/empty.rs | 4 - .../physical-plan/src/execution_plan.rs | 19 ++-- datafusion/physical-plan/src/memory.rs | 3 - .../physical-plan/src/placeholder_row.rs | 8 -- datafusion/physical-plan/src/streaming.rs | 8 -- datafusion/physical-plan/src/work_table.rs | 8 -- 10 files changed, 36 insertions(+), 117 deletions(-) diff --git a/datafusion/core/tests/user_defined/insert_operation.rs b/datafusion/core/tests/user_defined/insert_operation.rs index 94523fb372ea..2c90abeb8047 100644 --- a/datafusion/core/tests/user_defined/insert_operation.rs +++ b/datafusion/core/tests/user_defined/insert_operation.rs @@ -180,10 +180,6 @@ impl ExecutionPlan for TestInsertExec { unimplemented!("TestInsertExec is a stub for testing.") } - fn yields_cooperatively(&self) -> bool { - true - } - fn with_cooperative_yields(self: Arc) -> Option> { Some(self) } diff --git a/datafusion/datasource/src/sink.rs b/datafusion/datasource/src/sink.rs index 6a2f36f65854..7a94208a1a32 100644 --- a/datafusion/datasource/src/sink.rs +++ b/datafusion/datasource/src/sink.rs @@ -249,10 +249,6 @@ impl ExecutionPlan for DataSinkExec { self.sink.metrics() } - fn yields_cooperatively(&self) -> bool { - true - } - fn with_cooperative_yields(self: Arc) -> Option> { Some(self) } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 1722f3a8af69..864e9b9037ce 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -278,14 +278,6 @@ impl ExecutionPlan for DataSourceExec { Ok(Box::pin(yielding_stream)) } - /// Override: this operator *does* support cooperative yielding when `cooperative == true`. - fn yields_cooperatively(&self) -> bool { - self.cooperative - } - - /// If `cooperative == true`, return `Some(self.clone())` so the optimizer knows - /// we can replace a plain DataSourceExec with this same node (it already yields). - /// Otherwise, return None. fn with_cooperative_yields(self: Arc) -> Option> { self.cooperative.then_some(self) } diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index ca25a401801b..89041e2ad5bb 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -15,63 +15,32 @@ // specific language governing permissions and limitations // under the License. +//! The `WrapLeaves` optimizer rule inspects the physical plan to look for +//! tight-looping operators and inserts explicit yielding mechanisms (whether +//! as a separate operator, or via a yielding variant) at leaf nodes to make +//! the plan cancellation friendly. + +use std::fmt::{Debug, Formatter}; +use std::sync::Arc; + use crate::PhysicalOptimizerRule; + use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion}; use datafusion_common::Result; use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::ExecutionPlan; -use std::fmt::{Debug, Formatter}; -use std::sync::Arc; -/// WrapLeaves is a PhysicalOptimizerRule that finds every *leaf* node in the -/// entire plan, and replaces it with a variant that can cooperatively yield -/// (either using its built‐in `with_cooperative_yields()` or, if none exists, -/// by wrapping it in a `YieldStreamExec` wrapper). -/// -/// In contrast to the previous behavior (which only looked at “Final”/pipeline‐ -/// breaking nodes), this modified rule simply wraps *every* leaf no matter what. +/// `WrapLeaves` is a [`PhysicalOptimizerRule`] that finds every leaf node in +/// the plan, and replaces it with a variant that cooperatively yields +/// either using the its yielding variant given by `with_cooperative_yields`, +/// or, if none exists, by inserting a [`YieldStreamExec`] operator as a parent. pub struct WrapLeaves {} impl WrapLeaves { pub fn new() -> Self { Self {} } - - /// Called when we encounter any node during `transform_down()`. If the node - /// has no children, it is a leaf. We check if it has a built‐in cooperative - /// yield variant (`with_cooperative_yields()`); if so, we replace it with that. - /// Otherwise, we wrap it in a `YieldStreamExec`. - /// - /// We then return `TreeNodeRecursion::Jump` so that we do not attempt to go - /// deeper under this node (there are no children, anyway). - fn wrap_leaves( - plan: Arc, - yield_frequency: usize, - ) -> Result>> { - if plan.children().is_empty() { - // This is a leaf. Try to see if the plan itself has a cooperative‐yield variant. - if let Some(coop_variant) = Arc::clone(&plan).with_cooperative_yields() { - // Replace with the built‐in cooperative yield version. - Ok(Transformed::new( - coop_variant, - /* changed= */ true, - TreeNodeRecursion::Jump, - )) - } else { - // Otherwise wrap it in a YieldStreamExec to enforce periodic yielding. - let wrapped = Arc::new(YieldStreamExec::new(plan, yield_frequency)); - Ok(Transformed::new( - wrapped, - /* changed= */ true, - TreeNodeRecursion::Jump, - )) - } - } else { - // Not a leaf: leave unchanged for now, keep recursing down. - Ok(Transformed::no(plan)) - } - } } impl Default for WrapLeaves { @@ -96,19 +65,23 @@ impl PhysicalOptimizerRule for WrapLeaves { plan: Arc, config: &ConfigOptions, ) -> Result> { - // Only activate if user has configured a nonzero yield frequency. - if config.optimizer.yield_frequency_for_pipeline_break != 0 { - let yield_frequency = config.optimizer.yield_frequency_for_pipeline_break; - - // We perform a single top‐level transform_down over the entire plan. - // For each node encountered, we call `wrap_leaves`. If the node is - // a leaf, it will be replaced with a yielding variant (either its - // built‐in cooperative version or an explicit YieldStreamExec). - let new_plan = plan.transform_down(|node: Arc| { - Self::wrap_leaves(node, yield_frequency) - })?; - - Ok(new_plan.data) + // Only activate if user has configured a non-zero yield frequency. + let yield_frequency = config.optimizer.yield_frequency_for_pipeline_break; + if yield_frequency != 0 { + plan.transform_down(|plan| { + if !plan.children().is_empty() { + // Not a leaf, keep recursing down. + return Ok(Transformed::no(plan)); + } + let new_plan = Arc::clone(&plan) + .with_cooperative_yields() + .unwrap_or_else(|| { + // Otherwise, insert a `YieldStreamExec` to enforce periodic yielding. + Arc::new(YieldStreamExec::new(plan, yield_frequency)) + }); + Ok(Transformed::new(new_plan, true, TreeNodeRecursion::Jump)) + }) + .map(|t| t.data) } else { // If yield_frequency is zero, we do nothing. Ok(plan) diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 7507e05738da..68f9eb22e330 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -174,10 +174,6 @@ impl ExecutionPlan for EmptyExec { )) } - fn yields_cooperatively(&self) -> bool { - true - } - fn with_cooperative_yields(self: Arc) -> Option> { Some(self) } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 2641b1f47251..d9c78f448d98 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -547,20 +547,13 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { )) } - /// Whether this operator supports cooperative yielding. Default is false. - fn yields_cooperatively(&self) -> bool { - false - } - - /// If `yields_cooperatively() == true`, return an Arc to a new version - /// of this plan node that includes built‐in yielding. Otherwise, return None. - /// - /// We have removed any `Self: Sized` bound so that this method can be - /// invoked on a trait object (`Arc`). The default - /// implementation simply returns `None`. Concrete types (e.g. DataSourceExec, LazyMemoryExec) - /// should override this method when they really do have a built‐in yielding variant. + /// Returns a version of this plan that cooperates with the runtime via + /// built‐in yielding. If such a version doesn't exist, returns `None`. + /// You do not need to do provide such a version of a custom operator, + /// but DataFusion will utilize it while optimizing the plan if it exists. fn with_cooperative_yields(self: Arc) -> Option> { - // Default: no built‐in yielding, so return None. + // Conservative default implementation assumes that a leaf does not + // cooperate with yielding. None } } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 60117042c3e7..1590042bfb95 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -294,9 +294,6 @@ impl ExecutionPlan for LazyMemoryExec { Ok(Box::pin(yielding_stream)) } - /// If `cooperative == true`, return `Some(self.clone())` so the optimizer knows - /// we can replace a plain DataSourceExec with this same node (it already yields). - /// Otherwise, return None. fn with_cooperative_yields(self: Arc) -> Option> { self.cooperative.then_some(self) } diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 2e21fdd569d7..aff76d708f0e 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -201,14 +201,6 @@ impl ExecutionPlan for PlaceholderRowExec { )) } - /// Override: this operator *does* support cooperative yielding when `cooperative == true`. - fn yields_cooperatively(&self) -> bool { - self.cooperative - } - - /// If `cooperative == true`, return `Some(self.clone())` so the optimizer knows - /// we can replace a plain DataSourceExec with this same node (it already yields). - /// Otherwise, return None. fn with_cooperative_yields(self: Arc) -> Option> { self.cooperative.then_some(self) } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 7495c03029b3..b95e54e345bc 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -379,14 +379,6 @@ impl ExecutionPlan for StreamingTableExec { })) } - /// Override: this operator *does* support cooperative yielding when `cooperative == true`. - fn yields_cooperatively(&self) -> bool { - self.cooperative - } - - /// If `cooperative == true`, return `Some(self.clone())` so the optimizer knows - /// we can replace a plain DataSourceExec with this same node (it already yields). - /// Otherwise, return None. fn with_cooperative_yields(self: Arc) -> Option> { self.cooperative.then_some(self) } diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index e60fb58d94a8..4f07ecb60661 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -254,14 +254,6 @@ impl ExecutionPlan for WorkTableExec { Ok(Statistics::new_unknown(&self.schema())) } - /// Override: this operator *does* support cooperative yielding when `cooperative == true`. - fn yields_cooperatively(&self) -> bool { - self.cooperative - } - - /// If `cooperative == true`, return `Some(self.clone())` so the optimizer knows - /// we can replace a plain DataSourceExec with this same node (it already yields). - /// Otherwise, return None. fn with_cooperative_yields(self: Arc) -> Option> { self.cooperative.then_some(self) } From c820e363b369ddfff688cce258dd7fbd636c6621 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 6 Jun 2025 23:11:35 +0300 Subject: [PATCH 68/74] Format imports --- datafusion/physical-plan/src/memory.rs | 3 +-- datafusion/physical-plan/src/placeholder_row.rs | 12 +++++++----- datafusion/proto/src/physical_plan/mod.rs | 10 +++++----- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 1590042bfb95..dd4072082323 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -24,6 +24,7 @@ use std::task::{Context, Poll}; use crate::execution_plan::{Boundedness, EmissionType}; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::yield_stream::YieldStream; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -36,7 +37,6 @@ use datafusion_execution::memory_pool::MemoryReservation; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; -use crate::yield_stream::YieldStream; use futures::Stream; use parking_lot::RwLock; @@ -166,7 +166,6 @@ impl LazyMemoryExec { EmissionType::Incremental, Boundedness::Bounded, ); - Ok(Self { schema, batch_generators: generators, diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index aff76d708f0e..1d2d8842891c 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -22,16 +22,18 @@ use std::sync::Arc; use crate::execution_plan::{Boundedness, EmissionType}; use crate::memory::MemoryStream; -use crate::{common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics}; -use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; -use arrow::array::{ArrayRef, NullArray}; -use arrow::array::{RecordBatch, RecordBatchOptions}; +use crate::yield_stream::YieldStream; +use crate::{ + common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, + SendableRecordBatchStream, Statistics, +}; + +use arrow::array::{ArrayRef, NullArray, RecordBatch, RecordBatchOptions}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; -use crate::yield_stream::YieldStream; use log::trace; /// Execution plan for empty relation with produce_one_row=true diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index a6037a400145..ef18c486275a 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -310,7 +310,6 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { runtime, extension_codec, ), - #[cfg_attr(not(feature = "parquet"), allow(unused_variables))] PhysicalPlanType::ParquetSink(sink) => self .try_into_parquet_sink_physical_plan( @@ -325,7 +324,6 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { runtime, extension_codec, ), - PhysicalPlanType::YieldStream(yield_stream) => self .try_into_yield_stream_physical_plan( yield_stream, @@ -1794,8 +1792,10 @@ impl protobuf::PhysicalPlanNode { ) -> Result> { let input = into_physical_plan(&field_stream.input, registry, runtime, extension_codec)?; - let frequency = field_stream.frequency as usize; - Ok(Arc::new(YieldStreamExec::new(input, frequency))) + Ok(Arc::new(YieldStreamExec::new( + input, + field_stream.frequency as _, + ))) } fn try_from_explain_exec( @@ -2793,7 +2793,7 @@ impl protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::YieldStream(Box::new( protobuf::YieldStreamExecNode { input: Some(Box::new(input)), - frequency: frequency as u32, + frequency: frequency as _, }, ))), }) From 697bfae01596d6e1b6bd680d1b94377e1f0a9654 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 6 Jun 2025 23:19:44 +0300 Subject: [PATCH 69/74] Remove unnecessary clones --- .../core/tests/execution/infinite_cancel.rs | 17 +++++++---------- datafusion/proto/src/physical_plan/mod.rs | 4 +--- 2 files changed, 8 insertions(+), 13 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index e8884e7f947b..db2f6c4cd589 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -573,8 +573,8 @@ async fn test_infinite_join_agg_cancel( let coalesced_left = Arc::new(CoalesceBatchesExec::new(infinite_left, 8_192)); let coalesced_right = Arc::new(CoalesceBatchesExec::new(infinite_right, 8_192)); - let part_left = Partitioning::Hash(left_keys.clone(), 1); - let part_right = Partitioning::Hash(right_keys.clone(), 1); + let part_left = Partitioning::Hash(left_keys, 1); + let part_right = Partitioning::Hash(right_keys, 1); let hashed_left = Arc::new(RepartitionExec::try_new(coalesced_left, part_left)?); let hashed_right = Arc::new(RepartitionExec::try_new(coalesced_right, part_right)?); @@ -742,8 +742,8 @@ async fn test_infinite_hash_join_without_repartition_and_no_agg( // 2c) Directly feed `coalesced_left` and `coalesced_right` into HashJoinExec. // Do not use aggregation or repartition. let join = Arc::new(HashJoinExec::try_new( - coalesced_left.clone(), - coalesced_right.clone(), + coalesced_left, + coalesced_right, vec![( Arc::new(Column::new_with_schema("value", &schema)?), Arc::new(Column::new_with_schema("value", &schema)?), @@ -827,12 +827,9 @@ async fn test_infinite_sort_merge_join_without_repartition_and_no_agg( // We need a Vec for the join key. Any consistent SortOptions works, // because data is already in ascending order on “value.” let join = Arc::new(SortMergeJoinExec::try_new( - coalesced_left.clone(), - coalesced_right.clone(), - vec![( - col("value", &coalesced_left.schema())?, - col("value", &coalesced_right.schema())?, - )], + coalesced_left, + coalesced_right, + vec![(col("value", &schema)?, col("value", &schema)?)], /* filter */ None, JoinType::Inner, vec![SortOptions::new(true, false)], // ascending, nulls last diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index ef18c486275a..cb6943253e21 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -2787,13 +2787,11 @@ impl protobuf::PhysicalPlanNode { extension_codec, )?; - let frequency = exec.yield_frequency(); - Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::YieldStream(Box::new( protobuf::YieldStreamExecNode { input: Some(Box::new(input)), - frequency: frequency as _, + frequency: exec.yield_frequency() as _, }, ))), }) From d3952d9ef071c551c46f208efda0101974b97ac5 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 7 Jun 2025 10:13:24 +0300 Subject: [PATCH 70/74] Fix logical conflict --- datafusion/core/tests/execution/infinite_cancel.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index db2f6c4cd589..cce7a4b01b02 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -239,8 +239,7 @@ async fn test_infinite_sort_cancel( Arc::new(Column::new_with_schema("value", &schema)?), sort_options, ); - let lex_ordering: datafusion::physical_expr::LexOrdering = vec![sort_expr].into(); - let sort_exec = Arc::new(SortExec::new(lex_ordering, inf)); + let sort_exec = Arc::new(SortExec::new([sort_expr].into(), inf)); // 4) optimize the plan with WrapLeaves to auto-insert Yield let config = ConfigOptions::new(); From 56361a43d889b5003da9fe0b3c476f68f0d1f88e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 7 Jun 2025 11:12:06 +0300 Subject: [PATCH 71/74] Exercise DRY, refactor common code to a helper function. Use period instead of frequency for config parameter terminology --- datafusion/common/src/config.rs | 6 ++-- datafusion/datasource/src/source.rs | 22 +++--------- .../src/wrap_leaves_cancellation.rs | 7 ++-- datafusion/physical-plan/src/memory.rs | 18 ++-------- .../physical-plan/src/placeholder_row.rs | 23 ++---------- datafusion/physical-plan/src/streaming.rs | 35 +++---------------- datafusion/physical-plan/src/work_table.rs | 20 ++--------- datafusion/physical-plan/src/yield_stream.rs | 33 +++++++++++++---- datafusion/proto/src/physical_plan/mod.rs | 2 +- .../test_files/information_schema.slt | 4 +-- docs/source/user-guide/configs.md | 2 +- 11 files changed, 52 insertions(+), 120 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 30d1f1c7a545..4efb67a37c99 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -727,10 +727,10 @@ config_namespace! { /// due to the presence of tight-looping operators, it will attempt to /// mitigate this by inserting explicit yielding (in as few places as /// possible to avoid performance degradation). This value represents the - /// yielding frequency (in batches) at such explicit yielding points. - /// The default value is 64. If set to 0, no DataFusion will not perform + /// yielding period (in batches) at such explicit yielding points. The + /// default value is 64. If set to 0, no DataFusion will not perform /// any explicit yielding. - pub yield_frequency_for_pipeline_break: usize, default = 64 + pub yield_period: usize, default = 64 } } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 864e9b9037ce..29c3c9c3d7ff 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -38,7 +38,7 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::filter_pushdown::{ ChildPushdownResult, FilterPushdownPropagation, }; -use datafusion_physical_plan::yield_stream::YieldStream; +use datafusion_physical_plan::yield_stream::wrap_yield_stream; /// A source of data, typically a list of files or memory /// @@ -259,23 +259,9 @@ impl ExecutionPlan for DataSourceExec { partition: usize, context: Arc, ) -> Result { - // 1. Get the “base” stream exactly as before, without yielding. - let stream = self.data_source.open(partition, Arc::clone(&context)); - - // 2. If cooperative == false, return base_stream immediately. - if !self.cooperative { - return stream; - } - - let frequency = context - .session_config() - .options() - .optimizer - .yield_frequency_for_pipeline_break; - - // 3. If cooperative == true, wrap the stream into a YieldStream. - let yielding_stream = YieldStream::new(stream?, frequency); - Ok(Box::pin(yielding_stream)) + self.data_source + .open(partition, Arc::clone(&context)) + .map(|stream| wrap_yield_stream(stream, &context, self.cooperative)) } fn with_cooperative_yields(self: Arc) -> Option> { diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs index 89041e2ad5bb..95f84bd06396 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs @@ -66,8 +66,8 @@ impl PhysicalOptimizerRule for WrapLeaves { config: &ConfigOptions, ) -> Result> { // Only activate if user has configured a non-zero yield frequency. - let yield_frequency = config.optimizer.yield_frequency_for_pipeline_break; - if yield_frequency != 0 { + let yield_period = config.optimizer.yield_period; + if yield_period != 0 { plan.transform_down(|plan| { if !plan.children().is_empty() { // Not a leaf, keep recursing down. @@ -77,13 +77,12 @@ impl PhysicalOptimizerRule for WrapLeaves { .with_cooperative_yields() .unwrap_or_else(|| { // Otherwise, insert a `YieldStreamExec` to enforce periodic yielding. - Arc::new(YieldStreamExec::new(plan, yield_frequency)) + Arc::new(YieldStreamExec::new(plan, yield_period)) }); Ok(Transformed::new(new_plan, true, TreeNodeRecursion::Jump)) }) .map(|t| t.data) } else { - // If yield_frequency is zero, we do nothing. Ok(plan) } } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index dd4072082323..4489e5224188 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -24,7 +24,7 @@ use std::task::{Context, Poll}; use crate::execution_plan::{Boundedness, EmissionType}; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use crate::yield_stream::YieldStream; +use crate::yield_stream::wrap_yield_stream; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -276,21 +276,7 @@ impl ExecutionPlan for LazyMemoryExec { generator: Arc::clone(&self.batch_generators[partition]), baseline_metrics, }); - - // 2. If cooperative == false, return base_stream immediately. - if !self.cooperative { - return Ok(stream); - } - - let frequency = context - .session_config() - .options() - .optimizer - .yield_frequency_for_pipeline_break; - - // 3. If cooperative == true, wrap the stream into a YieldStream. - let yielding_stream = YieldStream::new(stream, frequency); - Ok(Box::pin(yielding_stream)) + Ok(wrap_yield_stream(stream, &context, self.cooperative)) } fn with_cooperative_yields(self: Arc) -> Option> { diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 1d2d8842891c..a5c80438e774 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use crate::execution_plan::{Boundedness, EmissionType}; use crate::memory::MemoryStream; -use crate::yield_stream::YieldStream; +use crate::yield_stream::wrap_yield_stream; use crate::{ common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, @@ -164,25 +164,8 @@ impl ExecutionPlan for PlaceholderRowExec { ); } - let memory_stream = - MemoryStream::try_new(self.data()?, Arc::clone(&self.schema), None)?; - - let stream: SendableRecordBatchStream = Box::pin(memory_stream); - - if !self.cooperative { - return Ok(stream); - } - - let frequency = context - .session_config() - .options() - .optimizer - .yield_frequency_for_pipeline_break; - - let yielding = YieldStream::new(stream, frequency); - let yielding_stream: SendableRecordBatchStream = Box::pin(yielding); - - Ok(yielding_stream) + MemoryStream::try_new(self.data()?, Arc::clone(&self.schema), None) + .map(|ms| wrap_yield_stream(Box::pin(ms), &context, self.cooperative)) } fn statistics(&self) -> Result { diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 63c831c7346d..a3f593a06d72 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -30,7 +30,7 @@ use crate::projection::{ all_alias_free_columns, new_projections_for_columns, update_ordering, ProjectionExec, }; use crate::stream::RecordBatchStreamAdapter; -use crate::yield_stream::YieldStream; +use crate::yield_stream::wrap_yield_stream; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; use arrow::datatypes::{Schema, SchemaRef}; @@ -276,40 +276,13 @@ impl ExecutionPlan for StreamingTableExec { )), None => stream, }; - - if self.cooperative { - let frequency = ctx - .session_config() - .options() - .optimizer - .yield_frequency_for_pipeline_break; - - let yielding_stream = YieldStream::new(projected_stream, frequency); - - return Ok(match self.limit { - None => Box::pin(yielding_stream), - Some(fetch) => { - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); - Box::pin(LimitStream::new( - Box::pin(yielding_stream), - 0, - Some(fetch), - baseline_metrics, - )) - } - }); - } + let stream = wrap_yield_stream(projected_stream, &ctx, self.cooperative); Ok(match self.limit { - None => projected_stream, + None => stream, Some(fetch) => { let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); - Box::pin(LimitStream::new( - projected_stream, - 0, - Some(fetch), - baseline_metrics, - )) + Box::pin(LimitStream::new(stream, 0, Some(fetch), baseline_metrics)) } }) } diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index 4f07ecb60661..a5f094ffaf04 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -23,7 +23,7 @@ use std::sync::{Arc, Mutex}; use crate::execution_plan::{Boundedness, EmissionType}; use crate::memory::MemoryStream; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::yield_stream::YieldStream; +use crate::yield_stream::wrap_yield_stream; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SendableRecordBatchStream, Statistics, @@ -220,26 +220,12 @@ impl ExecutionPlan for WorkTableExec { } let batch = self.work_table.take()?; - // 1. Get the “base” stream exactly as before, without yielding. let stream = Box::pin( MemoryStream::try_new(batch.batches, Arc::clone(&self.schema), None)? .with_reservation(batch.reservation), ); - - // 2. If cooperative == false, return base_stream immediately. - if !self.cooperative { - return Ok(stream); - } - - let frequency = context - .session_config() - .options() - .optimizer - .yield_frequency_for_pipeline_break; - - // 3. If cooperative == true, wrap the stream into a YieldStream. - let yielding_stream = YieldStream::new(stream, frequency); - Ok(Box::pin(yielding_stream)) + // Cooperatively yield if asked to do so: + Ok(wrap_yield_stream(stream, &context, self.cooperative)) } fn metrics(&self) -> Option { diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index a99246655f85..4c645216b379 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -33,21 +33,24 @@ use datafusion_execution::TaskContext; use futures::{Stream, StreamExt}; /// An identity stream that passes batches through as is, but yields control -/// back to the runtime every `frequency` batches. This stream is useful to +/// back to the runtime every `period` batches. This stream is useful to /// construct a mechanism that allows operators that do not directly cooperate /// with the runtime to check/support cancellation. pub struct YieldStream { inner: SendableRecordBatchStream, batches_processed: usize, - frequency: usize, + period: usize, } impl YieldStream { - pub fn new(inner: SendableRecordBatchStream, frequency: usize) -> Self { + pub fn new(inner: SendableRecordBatchStream, mut period: usize) -> Self { + if period == 0 { + period = usize::MAX; + } Self { inner, batches_processed: 0, - frequency, + period, } } } @@ -59,7 +62,7 @@ impl Stream for YieldStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - if self.batches_processed >= self.frequency { + if self.batches_processed >= self.period { self.batches_processed = 0; cx.waker().wake_by_ref(); return Poll::Pending; @@ -113,9 +116,9 @@ impl YieldStreamExec { &self.child } - /// Returns the frequency at which the operator yields control back to the + /// Returns the period at which the operator yields control back to the /// runtime. - pub fn yield_frequency(&self) -> usize { + pub fn yield_period(&self) -> usize { self.frequency } } @@ -191,6 +194,22 @@ impl ExecutionPlan for YieldStreamExec { } } +/// Wraps `stream` inside a `YieldStream` depending on the `cooperative` flag. +/// Yielding period is extracted from `context`. +pub fn wrap_yield_stream( + mut stream: SendableRecordBatchStream, + context: &TaskContext, + cooperative: bool, +) -> SendableRecordBatchStream { + if cooperative { + let period = context.session_config().options().optimizer.yield_period; + if period > 0 { + stream = Box::pin(YieldStream::new(stream, period)); + } + } + stream +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index e44577d90e3e..e2c391d0445f 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -2788,7 +2788,7 @@ impl protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::YieldStream(Box::new( protobuf::YieldStreamExecNode { input: Some(Box::new(input)), - frequency: exec.yield_frequency() as _, + frequency: exec.yield_period() as _, }, ))), }) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index e45b69e549de..dc8b7680d83e 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -303,7 +303,7 @@ datafusion.optimizer.repartition_sorts true datafusion.optimizer.repartition_windows true datafusion.optimizer.skip_failed_rules false datafusion.optimizer.top_down_join_key_reordering true -datafusion.optimizer.yield_frequency_for_pipeline_break 64 +datafusion.optimizer.yield_period 64 datafusion.sql_parser.collect_spans false datafusion.sql_parser.dialect generic datafusion.sql_parser.enable_ident_normalization true @@ -414,7 +414,7 @@ datafusion.optimizer.repartition_sorts true Should DataFusion execute sorts in a datafusion.optimizer.repartition_windows true Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level datafusion.optimizer.skip_failed_rules false When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail datafusion.optimizer.top_down_join_key_reordering true When set to true, the physical plan optimizer will run a top down process to reorder the join keys -datafusion.optimizer.yield_frequency_for_pipeline_break 64 When DataFusion detects that a plan might not be promply cancellable due to the presence of tight-looping operators, it will attempt to mitigate this by inserting explicit yielding (in as few places as possible to avoid performance degradation). This value represents the yielding frequency (in batches) at such explicit yielding points. The default value is 64. If set to 0, no DataFusion will not perform any explicit yielding. +datafusion.optimizer.yield_period 64 When DataFusion detects that a plan might not be promply cancellable due to the presence of tight-looping operators, it will attempt to mitigate this by inserting explicit yielding (in as few places as possible to avoid performance degradation). This value represents the yielding period (in batches) at such explicit yielding points. The default value is 64. If set to 0, no DataFusion will not perform any explicit yielding. datafusion.sql_parser.collect_spans false When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. datafusion.sql_parser.dialect generic Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. datafusion.sql_parser.enable_ident_normalization true When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index abd9454a1e4a..42282e39e41f 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -119,7 +119,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | | datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | | datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | -| datafusion.optimizer.yield_frequency_for_pipeline_break | 64 | When DataFusion detects that a plan might not be promply cancellable due to the presence of tight-looping operators, it will attempt to mitigate this by inserting explicit yielding (in as few places as possible to avoid performance degradation). This value represents the yielding frequency (in batches) at such explicit yielding points. The default value is 64. If set to 0, no DataFusion will not perform any explicit yielding. | +| datafusion.optimizer.yield_period | 64 | When DataFusion detects that a plan might not be promply cancellable due to the presence of tight-looping operators, it will attempt to mitigate this by inserting explicit yielding (in as few places as possible to avoid performance degradation). This value represents the yielding period (in batches) at such explicit yielding points. The default value is 64. If set to 0, no DataFusion will not perform any explicit yielding. | | datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | | datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | | datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | From 50959ad01eb8a8d1ae96e779fd83fbd19b367ab9 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Mon, 9 Jun 2025 17:06:38 +0800 Subject: [PATCH 72/74] Update datafusion/physical-plan/src/yield_stream.rs Co-authored-by: Andrew Lamb --- datafusion/physical-plan/src/yield_stream.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion/physical-plan/src/yield_stream.rs b/datafusion/physical-plan/src/yield_stream.rs index 4c645216b379..0069b4b64d38 100644 --- a/datafusion/physical-plan/src/yield_stream.rs +++ b/datafusion/physical-plan/src/yield_stream.rs @@ -93,6 +93,11 @@ impl RecordBatchStream for YieldStream { /// batches. This is useful for operators that do not natively support yielding /// control, allowing them to be used in a runtime that requires yielding for /// cancellation or other purposes. +/// +/// # Note +/// If your ExecutionPlan periodically yields control back to the scheduler +/// implement [`ExecutionPlan::with_cooperative_yields`] to avoid the need for this +/// node. #[derive(Debug)] pub struct YieldStreamExec { /// The child execution plan that this operator "wraps" to make it From edc018d37b34100f73249ec466b52b8d6a53a000 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 9 Jun 2025 17:35:04 +0800 Subject: [PATCH 73/74] Address new comments --- .../core/tests/execution/infinite_cancel.rs | 40 +++++++++---------- ...s_cancellation.rs => insert_yield_exec.rs} | 18 ++++----- datafusion/physical-optimizer/src/lib.rs | 2 +- .../physical-optimizer/src/optimizer.rs | 4 +- .../sqllogictest/test_files/explain.slt | 6 +-- 5 files changed, 35 insertions(+), 35 deletions(-) rename datafusion/physical-optimizer/src/{wrap_leaves_cancellation.rs => insert_yield_exec.rs} (87%) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index cce7a4b01b02..f72e0b3acfa2 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -44,7 +44,7 @@ use datafusion_expr_common::operator::Operator::Gt; use datafusion_physical_expr::expressions::{col, BinaryExpr, Column, Literal}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; -use datafusion_physical_optimizer::wrap_leaves_cancellation::WrapLeaves; +use datafusion_physical_optimizer::insert_yield_exec::InsertYieldExec; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::filter::FilterExec; @@ -188,9 +188,9 @@ async fn test_infinite_agg_cancel( schema, )?); - // 3) optimize the plan with WrapLeaves to auto-insert Yield + // 3) optimize the plan with InsertYieldExec to auto-insert Yield let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(aggr, &config)?; + let optimized = InsertYieldExec::new().optimize(aggr, &config)?; // 4) get the stream let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; @@ -241,9 +241,9 @@ async fn test_infinite_sort_cancel( ); let sort_exec = Arc::new(SortExec::new([sort_expr].into(), inf)); - // 4) optimize the plan with WrapLeaves to auto-insert Yield + // 4) optimize the plan with InsertYieldExec to auto-insert Yield let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(sort_exec, &config)?; + let optimized = InsertYieldExec::new().optimize(sort_exec, &config)?; // 5) get the stream let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; @@ -322,13 +322,13 @@ async fn test_infinite_interleave_cancel( let filtered_interleave = Arc::new(FilterExec::try_new(always_false, interleave)?); // 5) Coalesce the filtered interleave into 8192-row batches. - // This lets WrapLeaves insert YieldStreamExec at each batch boundary. + // This lets InsertYieldExec insert YieldStreamExec at each batch boundary. let coalesced_top = Arc::new(CoalesceBatchesExec::new(filtered_interleave, 8192)); - // 6) Apply WrapLeaves to insert YieldStreamExec under every leaf. + // 6) Apply InsertYieldExec to insert YieldStreamExec under every leaf. // Each InfiniteExec → FilterExec → CoalesceBatchesExec chain will yield periodically. let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(coalesced_top, &config)?; + let optimized = InsertYieldExec::new().optimize(coalesced_top, &config)?; // 7) Execute the optimized plan with a 1-second timeout. // Because the top-level FilterExec always discards rows and the inputs are infinite, @@ -425,11 +425,11 @@ async fn test_infinite_interleave_agg_cancel( interleave_schema, )?); - // 5) WrapLeaves will automatically insert YieldStreams beneath each “infinite” leaf. + // 5) InsertYieldExec will automatically insert YieldStreams beneath each “infinite” leaf. // That way, each InfiniteExec (through the FilterExec/CoalesceBatchesExec/RepartitionExec chain) // yields to the runtime periodically instead of spinning CPU. let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(aggr, &config)?; + let optimized = InsertYieldExec::new().optimize(aggr, &config)?; // 6) Execute the stream. Because AggregateExec(mode=Single) only emits a final batch // after all inputs finish—and those inputs are infinite—we expect no output @@ -514,7 +514,7 @@ async fn test_infinite_join_cancel( // 3) Wrap yields under each infinite leaf let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(join, &config)?; + let optimized = InsertYieldExec::new().optimize(join, &config)?; // 4) Execute + 1 sec timeout let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; @@ -630,7 +630,7 @@ async fn test_infinite_join_agg_cancel( // 5) Wrap yields under each infinite leaf let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(aggr, &config)?; + let optimized = InsertYieldExec::new().optimize(aggr, &config)?; // 6) Execute + 1 sec timeout let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; @@ -682,9 +682,9 @@ async fn test_filter_reject_all_batches_cancel( // 2c) Use CoalesceBatchesExec to guarantee each Filter pull always yields an 8192-row batch let coalesced = Arc::new(CoalesceBatchesExec::new(filtered, 8_192)); - // 3) WrapLeaves to insert YieldExec—so that the InfiniteExec yields control between batches + // 3) InsertYieldExec to insert YieldExec—so that the InfiniteExec yields control between batches let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(coalesced, &config)?; + let optimized = InsertYieldExec::new().optimize(coalesced, &config)?; // 4) Execute with a 1-second timeout. Because Filter discards all 8192 rows each time // without ever producing output, no batch will arrive within 1 second. And since @@ -755,13 +755,13 @@ async fn test_infinite_hash_join_without_repartition_and_no_agg( /* build_left */ true, )?); - // 3) Do not apply WrapLeaves—since there is no aggregation, WrapLeaves would - // not insert a 'final' yield wrapper for the Join. If you want to skip WrapLeaves + // 3) Do not apply InsertYieldExec—since there is no aggregation, InsertYieldExec would + // not insert a 'final' yield wrapper for the Join. If you want to skip InsertYieldExec // entirely, comment out the next line; however, not calling it is equivalent // because there is no aggregation so no wrapper is inserted. Here we simply do - // not call WrapLeaves, ensuring the plan has neither aggregation nor repartition. + // not call InsertYieldExec, ensuring the plan has neither aggregation nor repartition. let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(join, &config)?; + let optimized = InsertYieldExec::new().optimize(join, &config)?; // 4) Execute with a 1 second timeout let mut stream = physical_plan::execute_stream(optimized, session_ctx.task_ctx())?; @@ -835,9 +835,9 @@ async fn test_infinite_sort_merge_join_without_repartition_and_no_agg( /* null_equal */ true, )?); - // 3) Do not apply WrapLeaves (no aggregation, no repartition → no built-in yields). + // 3) Do not apply InsertYieldExec (no aggregation, no repartition → no built-in yields). let config = ConfigOptions::new(); - let optimized = WrapLeaves::new().optimize(join, &config)?; + let optimized = InsertYieldExec::new().optimize(join, &config)?; // 4) Execute with a 1-second timeout. Because both sides are infinite and never match, // the SortMergeJoin will never produce output within 1s. diff --git a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs b/datafusion/physical-optimizer/src/insert_yield_exec.rs similarity index 87% rename from datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs rename to datafusion/physical-optimizer/src/insert_yield_exec.rs index 95f84bd06396..30a01a67cc68 100644 --- a/datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs +++ b/datafusion/physical-optimizer/src/insert_yield_exec.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! The `WrapLeaves` optimizer rule inspects the physical plan to look for +//! The `InsertYieldExec` optimizer rule inspects the physical plan to look for //! tight-looping operators and inserts explicit yielding mechanisms (whether //! as a separate operator, or via a yielding variant) at leaf nodes to make //! the plan cancellation friendly. @@ -31,33 +31,33 @@ use datafusion_common::Result; use datafusion_physical_plan::yield_stream::YieldStreamExec; use datafusion_physical_plan::ExecutionPlan; -/// `WrapLeaves` is a [`PhysicalOptimizerRule`] that finds every leaf node in +/// `InsertYieldExec` is a [`PhysicalOptimizerRule`] that finds every leaf node in /// the plan, and replaces it with a variant that cooperatively yields /// either using the its yielding variant given by `with_cooperative_yields`, /// or, if none exists, by inserting a [`YieldStreamExec`] operator as a parent. -pub struct WrapLeaves {} +pub struct InsertYieldExec {} -impl WrapLeaves { +impl InsertYieldExec { pub fn new() -> Self { Self {} } } -impl Default for WrapLeaves { +impl Default for InsertYieldExec { fn default() -> Self { Self::new() } } -impl Debug for WrapLeaves { +impl Debug for InsertYieldExec { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.debug_struct("WrapLeaves").finish() + f.debug_struct("InsertYieldExec").finish() } } -impl PhysicalOptimizerRule for WrapLeaves { +impl PhysicalOptimizerRule for InsertYieldExec { fn name(&self) -> &str { - "wrap_leaves" + "insert_yield_exec" } fn optimize( diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index f9d868ad207a..f7b5bd584351 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -30,6 +30,7 @@ pub mod combine_partial_final_agg; pub mod enforce_distribution; pub mod enforce_sorting; pub mod filter_pushdown; +pub mod insert_yield_exec; pub mod join_selection; pub mod limit_pushdown; pub mod limited_distinct_aggregation; @@ -41,6 +42,5 @@ pub mod sanity_checker; pub mod topk_aggregation; pub mod update_aggr_exprs; pub mod utils; -pub mod wrap_leaves_cancellation; pub use optimizer::PhysicalOptimizerRule; diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index bab866820318..d5129cea9d4e 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -26,6 +26,7 @@ use crate::combine_partial_final_agg::CombinePartialFinalAggregate; use crate::enforce_distribution::EnforceDistribution; use crate::enforce_sorting::EnforceSorting; use crate::filter_pushdown::FilterPushdown; +use crate::insert_yield_exec::InsertYieldExec; use crate::join_selection::JoinSelection; use crate::limit_pushdown::LimitPushdown; use crate::limited_distinct_aggregation::LimitedDistinctAggregation; @@ -34,7 +35,6 @@ use crate::projection_pushdown::ProjectionPushdown; use crate::sanity_checker::SanityCheckPlan; use crate::topk_aggregation::TopKAggregation; use crate::update_aggr_exprs::OptimizeAggregateOrder; -use crate::wrap_leaves_cancellation::WrapLeaves; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; @@ -138,7 +138,7 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), - Arc::new(WrapLeaves::new()), + Arc::new(InsertYieldExec::new()), // The SanityCheckPlan rule checks whether the order and // distribution requirements of each node in the plan // is satisfied. It will also reject non-runnable query diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 8c9054d56c14..560648271070 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -242,7 +242,7 @@ physical_plan after OutputRequirements DataSourceExec: file_groups={1 group: [[W physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after wrap_leaves SAME TEXT AS ABOVE +physical_plan after insert_yield_exec SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] @@ -320,7 +320,7 @@ physical_plan after OutputRequirements physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(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 ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after wrap_leaves SAME TEXT AS ABOVE +physical_plan after insert_yield_exec SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan_with_schema DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] @@ -362,7 +362,7 @@ physical_plan after OutputRequirements physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: 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, file_type=parquet physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after wrap_leaves SAME TEXT AS ABOVE +physical_plan after insert_yield_exec SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan DataSourceExec: 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, file_type=parquet physical_plan_with_stats DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] From ee9bb89dad34550437e8c3e997e1d3995fba2587 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Mon, 9 Jun 2025 18:16:12 +0800 Subject: [PATCH 74/74] Address more comments. --- .../core/tests/execution/infinite_cancel.rs | 168 +++++++----------- datafusion/physical-plan/src/memory.rs | 4 + 2 files changed, 70 insertions(+), 102 deletions(-) diff --git a/datafusion/core/tests/execution/infinite_cancel.rs b/datafusion/core/tests/execution/infinite_cancel.rs index f72e0b3acfa2..00c1f6b4486f 100644 --- a/datafusion/core/tests/execution/infinite_cancel.rs +++ b/datafusion/core/tests/execution/infinite_cancel.rs @@ -15,29 +15,23 @@ // specific language governing permissions and limitations // under the License. -use std::any::Any; use std::error::Error; use std::fmt::Formatter; -use std::pin::Pin; use std::sync::Arc; -use std::task::{Context, Poll}; use arrow::array::{Array, Int64Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use arrow_schema::SortOptions; -use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; use datafusion::functions_aggregate::sum; use datafusion::physical_expr::aggregate::AggregateExprBuilder; -use datafusion::physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion::physical_expr::Partitioning; +use datafusion::physical_plan; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; -use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, -}; +use datafusion::physical_plan::execution_plan::Boundedness; +use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; -use datafusion::{common, physical_plan}; use datafusion_common::config::ConfigOptions; use datafusion_common::{JoinType, ScalarValue}; use datafusion_expr_common::operator::Operator::Gt; @@ -54,101 +48,63 @@ use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::union::InterleaveExec; -use futures::{Stream, StreamExt}; +use datafusion_physical_plan::memory::{LazyBatchGenerator, LazyMemoryExec}; +use futures::StreamExt; +use parking_lot::RwLock; use rstest::rstest; use tokio::select; -struct InfiniteStream { - batch: RecordBatch, - poll_count: usize, -} - -impl RecordBatchStream for InfiniteStream { - fn schema(&self) -> SchemaRef { - self.batch.schema() - } -} - -impl Stream for InfiniteStream { - type Item = common::Result; - - fn poll_next( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - ) -> Poll> { - self.poll_count += 1; - Poll::Ready(Some(Ok(self.batch.clone()))) - } -} - #[derive(Debug)] -struct InfiniteExec { +/// A batch generator that can produce either bounded or boundless infinite stream of the same RecordBatch. +struct InfiniteGenerator { + /// The RecordBatch to return on each call. batch: RecordBatch, - properties: PlanProperties, + /// How many batches have already been generated. + counter: usize, } -impl InfiniteExec { - fn new(batch: RecordBatch, pretend_finite: bool) -> Self { - let properties = PlanProperties::new( - EquivalenceProperties::new(batch.schema().clone()), - Partitioning::UnknownPartitioning(1), - EmissionType::Incremental, - if pretend_finite { - Boundedness::Bounded - } else { - Boundedness::Unbounded { - requires_infinite_memory: false, - } - }, - ); - InfiniteExec { batch, properties } +impl std::fmt::Display for InfiniteGenerator { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + // Display current counter + write!(f, "InfiniteGenerator(counter={})", self.counter) } } -impl DisplayAs for InfiniteExec { - fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!(f, "infinite") +impl LazyBatchGenerator for InfiniteGenerator { + /// Generate the next RecordBatch. + fn generate_next_batch(&mut self) -> datafusion_common::Result> { + // Increment the counter and return a clone of the batch + self.counter += 1; + Ok(Some(self.batch.clone())) } } -impl ExecutionPlan for InfiniteExec { - fn name(&self) -> &str { - "infinite" - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.batch.schema() - } +/// Build a LazyMemoryExec that yields either a finite or infinite stream depending on `pretend_finite`. +fn make_lazy_exec( + batch: RecordBatch, + schema: SchemaRef, + pretend_finite: bool, +) -> Arc { + let boundedness = if pretend_finite { + Boundedness::Bounded + } else { + Boundedness::Unbounded { + requires_infinite_memory: false, + } + }; - fn properties(&self) -> &PlanProperties { - &self.properties - } + // Instantiate the generator with the batch and limit + let gen = InfiniteGenerator { batch, counter: 0 }; - fn children(&self) -> Vec<&Arc> { - vec![] - } + // Wrap the generator in a trait object behind Arc> + let generator: Arc> = Arc::new(RwLock::new(gen)); - fn with_new_children( - self: Arc, - _children: Vec>, - ) -> common::Result> { - Ok(self.clone()) - } + // Create a LazyMemoryExec with one partition using our generator + let mut exec = LazyMemoryExec::try_new(schema, vec![generator]).unwrap(); + exec.set_boundedness(boundedness); - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> common::Result { - Ok(Box::pin(InfiniteStream { - batch: self.batch.clone(), - poll_count: 0, - })) - } + // Erase concrete type into a generic ExecutionPlan handle + Arc::new(exec) as Arc } #[rstest] @@ -170,7 +126,7 @@ async fn test_infinite_agg_cancel( let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(builder.finish())])?; // 2) set up the infinite source + aggregation - let inf = Arc::new(InfiniteExec::new(batch, pretend_finite)); + let inf = make_lazy_exec(batch.clone(), schema.clone(), pretend_finite); let aggr = Arc::new(AggregateExec::try_new( AggregateMode::Single, PhysicalGroupBy::new(vec![], vec![], vec![]), @@ -228,7 +184,7 @@ async fn test_infinite_sort_cancel( let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(array)])?; // 2) set up the infinite source - let inf = Arc::new(InfiniteExec::new(batch, pretend_finite)); + let inf = make_lazy_exec(batch.clone(), schema.clone(), pretend_finite); // 3) set up a SortExec that will never finish because input is infinite let sort_options = SortOptions { @@ -289,8 +245,8 @@ async fn test_infinite_interleave_cancel( let thresholds = (0..32).map(|i| 8191 - (i * 256) as i64); for thr in thresholds { - // 2a) Construct an InfiniteExec for the sample batch. - let inf = Arc::new(InfiniteExec::new(batch.clone(), pretend_finite)); + // 2a) Set up the infinite source + let inf = make_lazy_exec(batch.clone(), schema.clone(), pretend_finite); // 2b) Apply a FilterExec with predicate "value > thr". let filter_expr = Arc::new(BinaryExpr::new( @@ -373,7 +329,7 @@ async fn test_infinite_interleave_agg_cancel( for thr in thresholds { // 2a) One infinite exec: - let inf = Arc::new(InfiniteExec::new(batch.clone(), pretend_finite)); + let inf = make_lazy_exec(batch.clone(), schema.clone(), pretend_finite); // 2b) Apply a FilterExec: “value > thr”. let filter_expr = Arc::new(BinaryExpr::new( @@ -478,8 +434,10 @@ async fn test_infinite_join_cancel( RecordBatch::try_new(schema.clone(), vec![Arc::new(builder_right.finish())])?; // 2a) Build two InfiniteExecs (left and right) - let infinite_left = Arc::new(InfiniteExec::new(batch_left, pretend_finite)); - let infinite_right = Arc::new(InfiniteExec::new(batch_right, pretend_finite)); + let infinite_left = + make_lazy_exec(batch_left.clone(), schema.clone(), pretend_finite); + let infinite_right = + make_lazy_exec(batch_right.clone(), schema.clone(), pretend_finite); // 2b) Create Join keys → join on “value” = “value” let left_keys: Vec> = @@ -559,8 +517,10 @@ async fn test_infinite_join_agg_cancel( RecordBatch::try_new(schema.clone(), vec![Arc::new(builder_right.finish())])?; // 2a) Build two InfiniteExecs (left and right) - let infinite_left = Arc::new(InfiniteExec::new(batch_left, pretend_finite)); - let infinite_right = Arc::new(InfiniteExec::new(batch_right, pretend_finite)); + let infinite_left = + make_lazy_exec(batch_left.clone(), schema.clone(), pretend_finite); + let infinite_right = + make_lazy_exec(batch_right.clone(), schema.clone(), pretend_finite); // 2b) Create Join keys → join on “value” = “value” let left_keys: Vec> = @@ -669,7 +629,7 @@ async fn test_filter_reject_all_batches_cancel( let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(builder.finish())])?; // 2a) Wrap this batch in an InfiniteExec - let infinite = Arc::new(InfiniteExec::new(batch, pretend_finite)); + let infinite = make_lazy_exec(batch.clone(), schema.clone(), pretend_finite); // 2b) Construct a FilterExec that is always false: “value > 10000” (no rows pass) let false_predicate = Arc::new(BinaryExpr::new( @@ -730,8 +690,10 @@ async fn test_infinite_hash_join_without_repartition_and_no_agg( // 2a) Unlike the test with aggregation, keep this as a pure join— // use InfiniteExec to simulate an infinite stream - let infinite_left = Arc::new(InfiniteExec::new(batch_left, pretend_finite)); - let infinite_right = Arc::new(InfiniteExec::new(batch_right, pretend_finite)); + let infinite_left = + make_lazy_exec(batch_left.clone(), schema.clone(), pretend_finite); + let infinite_right = + make_lazy_exec(batch_right.clone(), schema.clone(), pretend_finite); // 2b) To feed a single batch into the Join, we can still use CoalesceBatchesExec, // but do NOT wrap it in a RepartitionExec @@ -812,8 +774,10 @@ async fn test_infinite_sort_merge_join_without_repartition_and_no_agg( let batch_right = RecordBatch::try_new(schema.clone(), vec![right_array])?; // 2a) Wrap each small batch in an InfiniteExec (pretend_finite toggles finite vs infinite) - let infinite_left = Arc::new(InfiniteExec::new(batch_left, pretend_finite)); - let infinite_right = Arc::new(InfiniteExec::new(batch_right, pretend_finite)); + let infinite_left = + make_lazy_exec(batch_left.clone(), schema.clone(), pretend_finite); + let infinite_right = + make_lazy_exec(batch_right.clone(), schema.clone(), pretend_finite); // 2b) Coalesce each InfiniteExec into a single 3-row batch at a time. // (Do NOT wrap in RepartitionExec.) diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 4489e5224188..91af03bf46df 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -181,6 +181,10 @@ impl LazyMemoryExec { self.cooperative = cooperative; self } + + pub fn set_boundedness(&mut self, boundedness: Boundedness) { + self.cache.boundedness = boundedness; + } } impl fmt::Debug for LazyMemoryExec {