From c2deee7ed41fb3a36681523b45e857657e686885 Mon Sep 17 00:00:00 2001 From: jizezhang Date: Thu, 13 Nov 2025 19:42:15 -0800 Subject: [PATCH 01/25] feat: allow custom caching via logical node --- datafusion/core/src/dataframe/mod.rs | 27 +++++--- datafusion/core/src/execution/context/mod.rs | 15 +++++ .../core/src/execution/session_state.rs | 37 ++++++++++- datafusion/core/src/test_util/mod.rs | 66 ++++++++++++++++++- datafusion/core/tests/dataframe/mod.rs | 32 +++++++-- 5 files changed, 161 insertions(+), 16 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 2b94bcf63b52..2ea928604590 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -56,6 +56,7 @@ use datafusion_common::{ ScalarValue, SchemaError, TableReference, UnnestOptions, }; use datafusion_expr::select_expr::SelectExpr; +use datafusion_expr::Extension; use datafusion_expr::{ case, dml::InsertOp, @@ -2344,14 +2345,24 @@ impl DataFrame { /// # } /// ``` pub async fn cache(self) -> Result { - let context = SessionContext::new_with_state((*self.session_state).clone()); - // The schema is consistent with the output - let plan = self.clone().create_physical_plan().await?; - let schema = plan.schema(); - let task_ctx = Arc::new(self.task_ctx()); - let partitions = collect_partitioned(plan, task_ctx).await?; - let mem_table = MemTable::try_new(schema, partitions)?; - context.read_table(Arc::new(mem_table)) + if let Some(cache_producer) = self.session_state.cache_producer() { + let node = cache_producer.create(self.plan)?; + let plan = LogicalPlan::Extension(Extension { node }); + Ok(Self { + session_state: self.session_state, + plan, + projection_requires_validation: self.projection_requires_validation, + }) + } else { + let context = SessionContext::new_with_state((*self.session_state).clone()); + // The schema is consistent with the output + let plan = self.clone().create_physical_plan().await?; + let schema = plan.schema(); + let task_ctx = Arc::new(self.task_ctx()); + let partitions = collect_partitioned(plan, task_ctx).await?; + let mem_table = MemTable::try_new(schema, partitions)?; + context.read_table(Arc::new(mem_table)) + } } /// Apply an alias to the DataFrame. diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index bca4cce9ba77..4bee9a22d316 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -468,6 +468,12 @@ impl SessionContext { self } + /// Register a [`CacheProducer`] to provide custom caching strategy + pub fn with_cache_producer(self, cache_producer: Arc) -> Self { + self.state.write().set_cache_producer(cache_producer); + self + } + /// Adds an optimizer rule to the end of the existing rules. /// /// See [`SessionState`] for more control of when the rule is applied. @@ -1884,6 +1890,15 @@ pub enum RegisterFunction { Table(String, Arc), } +/// Interface for applying a custom caching strategy. +/// Implement this trait and register via [`SessionState`] +/// to create a custom logical node for caching. +pub trait CacheProducer: Debug + Sync + Send { + /// Create a custom logical node for caching + /// given a logical plan (of DF to cache). + fn create(&self, plan: LogicalPlan) -> Result>; +} + /// Default implementation of [SerializerRegistry] that throws unimplemented error /// for all requests. #[derive(Debug)] diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index d7a66db28ac4..40709868c084 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -27,7 +27,9 @@ use crate::catalog::{CatalogProviderList, SchemaProvider, TableProviderFactory}; use crate::datasource::file_format::FileFormatFactory; #[cfg(feature = "sql")] use crate::datasource::provider_as_source; -use crate::execution::context::{EmptySerializerRegistry, FunctionFactory, QueryPlanner}; +use crate::execution::context::{ + CacheProducer, EmptySerializerRegistry, FunctionFactory, QueryPlanner, +}; use crate::execution::SessionStateDefaults; use crate::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner}; use arrow_schema::{DataType, FieldRef}; @@ -185,6 +187,7 @@ pub struct SessionState { /// It will be invoked on `CREATE FUNCTION` statements. /// thus, changing dialect o PostgreSql is required function_factory: Option>, + cache_producer: Option>, /// Cache logical plans of prepared statements for later execution. /// Key is the prepared statement name. prepared_plans: HashMap>, @@ -206,6 +209,7 @@ impl Debug for SessionState { .field("table_options", &self.table_options) .field("table_factories", &self.table_factories) .field("function_factory", &self.function_factory) + .field("cache_producer", &self.cache_producer) .field("expr_planners", &self.expr_planners); #[cfg(feature = "sql")] @@ -355,6 +359,16 @@ impl SessionState { self.function_factory.as_ref() } + /// Register a [`CacheProducer`] for custom caching strategy + pub fn set_cache_producer(&mut self, cache_producer: Arc) { + self.cache_producer = Some(cache_producer); + } + + /// Get the cache producer + pub fn cache_producer(&self) -> Option<&Arc> { + self.cache_producer.as_ref() + } + /// Get the table factories pub fn table_factories(&self) -> &HashMap> { &self.table_factories @@ -941,6 +955,7 @@ pub struct SessionStateBuilder { table_factories: Option>>, runtime_env: Option>, function_factory: Option>, + cache_producer: Option>, // fields to support convenience functions analyzer_rules: Option>>, optimizer_rules: Option>>, @@ -978,6 +993,7 @@ impl SessionStateBuilder { table_factories: None, runtime_env: None, function_factory: None, + cache_producer: None, // fields to support convenience functions analyzer_rules: None, optimizer_rules: None, @@ -1030,7 +1046,7 @@ impl SessionStateBuilder { table_factories: Some(existing.table_factories), runtime_env: Some(existing.runtime_env), function_factory: existing.function_factory, - + cache_producer: existing.cache_producer, // fields to support convenience functions analyzer_rules: None, optimizer_rules: None, @@ -1319,6 +1335,15 @@ impl SessionStateBuilder { self } + /// Set a [`CacheProducer`] for custom caching strategy + pub fn with_cache_producer( + mut self, + cache_producer: Option>, + ) -> Self { + self.cache_producer = cache_producer; + self + } + /// Register an `ObjectStore` to the [`RuntimeEnv`]. See [`RuntimeEnv::register_object_store`] /// for more details. /// @@ -1382,6 +1407,7 @@ impl SessionStateBuilder { table_factories, runtime_env, function_factory, + cache_producer, analyzer_rules, optimizer_rules, physical_optimizer_rules, @@ -1418,6 +1444,7 @@ impl SessionStateBuilder { table_factories: table_factories.unwrap_or_default(), runtime_env, function_factory, + cache_producer, prepared_plans: HashMap::new(), }; @@ -1621,6 +1648,11 @@ impl SessionStateBuilder { &mut self.function_factory } + /// Returns the cache producer + pub fn cache_producer(&mut self) -> &mut Option> { + &mut self.cache_producer + } + /// Returns the current analyzer_rules value pub fn analyzer_rules( &mut self, @@ -1659,6 +1691,7 @@ impl Debug for SessionStateBuilder { .field("table_options", &self.table_options) .field("table_factories", &self.table_factories) .field("function_factory", &self.function_factory) + .field("cache_producer", &self.cache_producer) .field("expr_planners", &self.expr_planners); #[cfg(feature = "sql")] let ret = ret.field("type_planner", &self.type_planner); diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 7149c5b0bd8c..0925b4900c1d 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -36,6 +36,7 @@ use crate::dataframe::DataFrame; use crate::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; use crate::datasource::{empty::EmptyTable, provider_as_source}; use crate::error::Result; +use crate::execution::context::CacheProducer; use crate::logical_expr::{LogicalPlanBuilder, UNNAMED_TABLE}; use crate::physical_plan::ExecutionPlan; use crate::prelude::{CsvReadOptions, SessionContext}; @@ -44,8 +45,11 @@ use crate::execution::SendableRecordBatchStream; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_catalog::Session; -use datafusion_common::TableReference; -use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; +use datafusion_common::{DFSchemaRef, TableReference}; +use datafusion_expr::{ + CreateExternalTable, Expr, LogicalPlan, SortExpr, TableType, + UserDefinedLogicalNodeCore, +}; use std::pin::Pin; use async_trait::async_trait; @@ -282,3 +286,61 @@ impl RecordBatchStream for BoundedStream { self.record_batch.schema() } } + +#[derive(Hash, Eq, PartialEq, PartialOrd, Debug)] +struct CacheNode { + input: LogicalPlan, +} + +impl UserDefinedLogicalNodeCore for CacheNode { + fn name(&self) -> &str { + "CachNode" + } + + fn inputs(&self) -> Vec<&LogicalPlan> { + vec![&self.input] + } + + fn schema(&self) -> &DFSchemaRef { + self.input.schema() + } + + fn expressions(&self) -> Vec { + vec![] + } + + fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "CacheNode") + } + + fn with_exprs_and_inputs( + &self, + _exprs: Vec, + inputs: Vec, + ) -> Result { + assert_eq!(inputs.len(), 1, "input size inconsistent"); + Ok(Self { + input: inputs[0].clone(), + }) + } +} + +#[derive(Debug)] +struct TestCacheProducer {} + +impl CacheProducer for TestCacheProducer { + fn create( + &self, + plan: LogicalPlan, + ) -> Result> { + Ok(Arc::new(CacheNode { input: plan })) + } +} + +/// Create a test table registered to a session context with an associated cache producer +pub async fn test_table_with_cache_producer() -> Result { + let ctx = SessionContext::new().with_cache_producer(Arc::new(TestCacheProducer {})); + let name = "aggregate_test_100"; + register_aggregate_csv(&ctx, name).await?; + ctx.table(name).await +} diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 8fb06941927c..f52b50d9c943 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -53,7 +53,7 @@ use url::Url; use datafusion::dataframe::{DataFrame, DataFrameWriteOptions}; use datafusion::datasource::MemTable; use datafusion::error::Result; -use datafusion::execution::context::SessionContext; +use datafusion::execution::context::{CacheProducer, SessionContext}; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::logical_expr::{ColumnarValue, Volatility}; use datafusion::prelude::{ @@ -61,7 +61,7 @@ use datafusion::prelude::{ }; use datafusion::test_util::{ parquet_test_data, populate_csv_partitions, register_aggregate_csv, test_table, - test_table_with_name, + test_table_with_cache_producer, test_table_with_name, }; use datafusion_catalog::TableProvider; use datafusion_common::test_util::{batches_to_sort_string, batches_to_string}; @@ -78,8 +78,9 @@ use datafusion_expr::var_provider::{VarProvider, VarType}; use datafusion_expr::{ cast, col, create_udf, exists, in_subquery, lit, out_ref_col, placeholder, scalar_subquery, when, wildcard, Expr, ExprFunctionExt, ExprSchemable, LogicalPlan, - LogicalPlanBuilder, ScalarFunctionImplementation, SortExpr, TableType, WindowFrame, - WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, + LogicalPlanBuilder, ScalarFunctionImplementation, SortExpr, TableType, + UserDefinedLogicalNodeCore, WindowFrame, WindowFrameBound, WindowFrameUnits, + WindowFunctionDefinition, }; use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::Column; @@ -2338,6 +2339,29 @@ async fn cache_test() -> Result<()> { Ok(()) } +#[tokio::test] +async fn cache_producer_test() -> Result<()> { + let df = test_table_with_cache_producer() + .await? + .select_columns(&["c2", "c3"])? + .limit(0, Some(1))? + .with_column("sum", cast(col("c2") + col("c3"), DataType::Int64))?; + + let cached_df = df.clone().cache().await?; + + assert_snapshot!( + cached_df.clone().into_optimized_plan().unwrap(), + @r###" + CacheNode + Projection: aggregate_test_100.c2, aggregate_test_100.c3, CAST(CAST(aggregate_test_100.c2 AS Int64) + CAST(aggregate_test_100.c3 AS Int64) AS Int64) AS sum + Projection: aggregate_test_100.c2, aggregate_test_100.c3 + Limit: skip=0, fetch=1 + TableScan: aggregate_test_100, fetch=1 + "### + ); + Ok(()) +} + #[tokio::test] async fn partition_aware_union() -> Result<()> { let left = test_table().await?.select_columns(&["c1", "c2"])?; From e83038a8cb4a966663652523f17e7d87cbb587b9 Mon Sep 17 00:00:00 2001 From: jizezhang Date: Thu, 13 Nov 2025 20:02:20 -0800 Subject: [PATCH 02/25] formatting --- datafusion/core/src/test_util/mod.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 0925b4900c1d..747f86dcb9be 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -25,6 +25,7 @@ pub mod csv; use futures::Stream; use std::any::Any; use std::collections::HashMap; +use std::fmt::Formatter; use std::fs::File; use std::io::Write; use std::path::Path; @@ -294,7 +295,7 @@ struct CacheNode { impl UserDefinedLogicalNodeCore for CacheNode { fn name(&self) -> &str { - "CachNode" + "CacheNode" } fn inputs(&self) -> Vec<&LogicalPlan> { @@ -309,7 +310,7 @@ impl UserDefinedLogicalNodeCore for CacheNode { vec![] } - fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + fn fmt_for_explain(&self, f: &mut Formatter) -> std::fmt::Result { write!(f, "CacheNode") } @@ -329,10 +330,7 @@ impl UserDefinedLogicalNodeCore for CacheNode { struct TestCacheProducer {} impl CacheProducer for TestCacheProducer { - fn create( - &self, - plan: LogicalPlan, - ) -> Result> { + fn create(&self, plan: LogicalPlan) -> Result> { Ok(Arc::new(CacheNode { input: plan })) } } From d9cd72582252916127e29d416fe018f89d8da74c Mon Sep 17 00:00:00 2001 From: jizezhang Date: Thu, 13 Nov 2025 20:11:42 -0800 Subject: [PATCH 03/25] fix import --- datafusion/core/src/test_util/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 747f86dcb9be..4171096ec6cc 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -48,7 +48,7 @@ use arrow::record_batch::RecordBatch; use datafusion_catalog::Session; use datafusion_common::{DFSchemaRef, TableReference}; use datafusion_expr::{ - CreateExternalTable, Expr, LogicalPlan, SortExpr, TableType, + CreateExternalTable, Expr, LogicalPlan, SortExpr, TableType, UserDefinedLogicalNode, UserDefinedLogicalNodeCore, }; use std::pin::Pin; From b17e5bb7dbd4969a60907b1f7937697e26519119 Mon Sep 17 00:00:00 2001 From: jizezhang Date: Thu, 13 Nov 2025 21:18:16 -0800 Subject: [PATCH 04/25] remove unused test imports --- datafusion/core/tests/dataframe/mod.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index f52b50d9c943..bb592d967517 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -53,7 +53,7 @@ use url::Url; use datafusion::dataframe::{DataFrame, DataFrameWriteOptions}; use datafusion::datasource::MemTable; use datafusion::error::Result; -use datafusion::execution::context::{CacheProducer, SessionContext}; +use datafusion::execution::context::SessionContext; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::logical_expr::{ColumnarValue, Volatility}; use datafusion::prelude::{ @@ -78,9 +78,8 @@ use datafusion_expr::var_provider::{VarProvider, VarType}; use datafusion_expr::{ cast, col, create_udf, exists, in_subquery, lit, out_ref_col, placeholder, scalar_subquery, when, wildcard, Expr, ExprFunctionExt, ExprSchemable, LogicalPlan, - LogicalPlanBuilder, ScalarFunctionImplementation, SortExpr, TableType, - UserDefinedLogicalNodeCore, WindowFrame, WindowFrameBound, WindowFrameUnits, - WindowFunctionDefinition, + LogicalPlanBuilder, ScalarFunctionImplementation, SortExpr, TableType, WindowFrame, + WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::Column; From 8ae150fc51a9a9727c1f3a07b3b6d4cb6adaa760 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Fri, 14 Nov 2025 16:05:31 +0800 Subject: [PATCH 05/25] minor: Use allow->expect to explicitly suppress Clippy lint checks (#18686) ## Which issue does this PR close? - Closes #. ## Rationale for this change When suppressing certain clippy lint violations, use `expect` instead of `allow` to ensure there is an actual lint violation. ## What changes are included in this PR? ## Are these changes tested? ## Are there any user-facing changes? --- datafusion/core/src/dataframe/mod.rs | 2 +- datafusion/core/src/physical_planner.rs | 2 +- datafusion/expr/src/execution_props.rs | 2 +- datafusion/expr/src/literal.rs | 6 +++--- datafusion/expr/src/logical_plan/plan.rs | 2 +- datafusion/expr/src/utils.rs | 2 +- datafusion/physical-expr/src/expressions/literal.rs | 2 +- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 2ea928604590..aaeb22f374fa 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2403,7 +2403,7 @@ impl DataFrame { /// # Ok(()) /// # } /// ``` - #[allow(clippy::needless_pass_by_value)] + #[expect(clippy::needless_pass_by_value)] pub fn fill_null( &self, value: ScalarValue, diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 154fd6f9cb05..6bcc45e9e465 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2246,7 +2246,7 @@ impl DefaultPhysicalPlanner { /// Optimize a physical plan by applying each physical optimizer, /// calling observer(plan, optimizer after each one) - #[allow(clippy::needless_pass_by_value)] + #[expect(clippy::needless_pass_by_value)] pub fn optimize_physical_plan( &self, plan: Arc, diff --git a/datafusion/expr/src/execution_props.rs b/datafusion/expr/src/execution_props.rs index fe20ed9331cb..acfcc61b7ece 100644 --- a/datafusion/expr/src/execution_props.rs +++ b/datafusion/expr/src/execution_props.rs @@ -102,7 +102,7 @@ impl ExecutionProps { } /// Returns the provider for the `var_type`, if any - #[allow(clippy::needless_pass_by_value)] + #[expect(clippy::needless_pass_by_value)] pub fn get_var_provider( &self, var_type: VarType, diff --git a/datafusion/expr/src/literal.rs b/datafusion/expr/src/literal.rs index c7345a455a76..09d8e9bb58b2 100644 --- a/datafusion/expr/src/literal.rs +++ b/datafusion/expr/src/literal.rs @@ -21,12 +21,12 @@ use crate::Expr; use datafusion_common::{metadata::FieldMetadata, ScalarValue}; /// Create a literal expression -#[allow(clippy::needless_pass_by_value)] +#[expect(clippy::needless_pass_by_value)] pub fn lit(n: T) -> Expr { n.lit() } -#[allow(clippy::needless_pass_by_value)] +#[expect(clippy::needless_pass_by_value)] pub fn lit_with_metadata(n: T, metadata: Option) -> Expr { let Some(metadata) = metadata else { return n.lit(); @@ -47,7 +47,7 @@ pub fn lit_with_metadata(n: T, metadata: Option) -> E } /// Create a literal timestamp expression -#[allow(clippy::needless_pass_by_value)] +#[expect(clippy::needless_pass_by_value)] pub fn lit_timestamp_nano(n: T) -> Expr { n.lit_timestamp_nano() } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index c70501f11020..d879a58b29ec 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -3484,7 +3484,7 @@ impl Aggregate { /// /// This method should only be called when you are absolutely sure that the schema being /// provided is correct for the aggregate. If in doubt, call [try_new](Self::try_new) instead. - #[allow(clippy::needless_pass_by_value)] + #[expect(clippy::needless_pass_by_value)] pub fn try_new_with_schema( input: Arc, group_expr: Vec, diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index b4e763cdf497..8e8483bc2a35 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -928,7 +928,7 @@ pub fn find_valid_equijoin_key_pair( /// round(Float64) /// round(Float32) /// ``` -#[allow(clippy::needless_pass_by_value)] +#[expect(clippy::needless_pass_by_value)] pub fn generate_signature_error_msg( func_name: &str, func_signature: Signature, diff --git a/datafusion/physical-expr/src/expressions/literal.rs b/datafusion/physical-expr/src/expressions/literal.rs index 359bfcefdbb5..1f3fefc60b7a 100644 --- a/datafusion/physical-expr/src/expressions/literal.rs +++ b/datafusion/physical-expr/src/expressions/literal.rs @@ -137,7 +137,7 @@ impl PhysicalExpr for Literal { } /// Create a literal expression -#[allow(clippy::needless_pass_by_value)] +#[expect(clippy::needless_pass_by_value)] pub fn lit(value: T) -> Arc { match value.lit() { Expr::Literal(v, _) => Arc::new(Literal::new(v)), From 13a8a3b2add865e167a38f7628df28079bbc7923 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 14 Nov 2025 20:03:50 +1100 Subject: [PATCH 06/25] chore(deps): bump taiki-e/install-action from 2.62.50 to 2.62.51 (#18693) Bumps [taiki-e/install-action](https://github.com/taiki-e/install-action) from 2.62.50 to 2.62.51.
Release notes

Sourced from taiki-e/install-action's releases.

2.62.51

  • Update typos@latest to 1.39.2.

  • Update mise@latest to 2025.11.4.

  • Update uv@latest to 0.9.9.

  • Update protoc@latest to 3.33.1.

  • Update just@latest to 1.43.1.

Changelog

Sourced from taiki-e/install-action's changelog.

Changelog

All notable changes to this project will be documented in this file.

This project adheres to Semantic Versioning.

[Unreleased]

[2.62.51] - 2025-11-14

  • Update typos@latest to 1.39.2.

  • Update mise@latest to 2025.11.4.

  • Update uv@latest to 0.9.9.

  • Update protoc@latest to 3.33.1.

  • Update just@latest to 1.43.1.

[2.62.50] - 2025-11-12

  • Update wasmtime@latest to 38.0.4.

  • Update coreutils@latest to 0.4.0.

[2.62.49] - 2025-11-09

  • Update cargo-binstall@latest to 1.15.11.

  • Update cargo-auditable@latest to 0.7.2.

  • Update vacuum@latest to 0.20.2.

[2.62.48] - 2025-11-08

  • Update mise@latest to 2025.11.3.

  • Update cargo-audit@latest to 0.22.0.

  • Update vacuum@latest to 0.20.1.

  • Update uv@latest to 0.9.8.

  • Update cargo-udeps@latest to 0.1.60.

... (truncated)

Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=taiki-e/install-action&package-manager=github_actions&previous-version=2.62.50&new-version=2.62.51)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/audit.yml | 2 +- .github/workflows/rust.yml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/audit.yml b/.github/workflows/audit.yml index 952c0dc7849d..13d9cdce355e 100644 --- a/.github/workflows/audit.yml +++ b/.github/workflows/audit.yml @@ -42,7 +42,7 @@ jobs: steps: - uses: actions/checkout@08c6903cd8c0fde910a37f88322edcfb5dd907a8 # v5.0.0 - name: Install cargo-audit - uses: taiki-e/install-action@6cc14f7f2f4b3129aff07a8b071d2d4f2733465d # v2.62.50 + uses: taiki-e/install-action@0be4756f42223b67aa4b7df5effad59010cbf4b9 # v2.62.51 with: tool: cargo-audit - name: Run audit check diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index d5c5e0ef4d7e..64f3a3257101 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -434,7 +434,7 @@ jobs: sudo apt-get update -qq sudo apt-get install -y -qq clang - name: Setup wasm-pack - uses: taiki-e/install-action@6cc14f7f2f4b3129aff07a8b071d2d4f2733465d # v2.62.50 + uses: taiki-e/install-action@0be4756f42223b67aa4b7df5effad59010cbf4b9 # v2.62.51 with: tool: wasm-pack - name: Run tests with headless mode @@ -761,7 +761,7 @@ jobs: - name: Setup Rust toolchain uses: ./.github/actions/setup-builder - name: Install cargo-msrv - uses: taiki-e/install-action@6cc14f7f2f4b3129aff07a8b071d2d4f2733465d # v2.62.50 + uses: taiki-e/install-action@0be4756f42223b67aa4b7df5effad59010cbf4b9 # v2.62.51 with: tool: cargo-msrv From bc887808faf15a8938934b682203c9796d0add30 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 14 Nov 2025 09:04:19 +0000 Subject: [PATCH 07/25] chore(deps): bump crate-ci/typos from 1.39.1 to 1.39.2 (#18694) Bumps [crate-ci/typos](https://github.com/crate-ci/typos) from 1.39.1 to 1.39.2.
Release notes

Sourced from crate-ci/typos's releases.

v1.39.2

[1.39.2] - 2025-11-13

Fixes

  • Don't offer entry as a correction for entrys
Changelog

Sourced from crate-ci/typos's changelog.

Change Log

All notable changes to this project will be documented in this file.

The format is based on Keep a Changelog and this project adheres to Semantic Versioning.

[Unreleased] - ReleaseDate

[1.39.2] - 2025-11-13

Fixes

  • Don't offer entry as a correction for entrys

[1.39.1] - 2025-11-12

Features

  • Make --help more vibrant

[1.39.0] - 2025-10-31

Features

Fixes

  • When a typo is pluralized, prefer pluralized corrections

[1.38.1] - 2025-10-07

Fixes

  • Ignore common golang identifiers

[1.38.0] - 2025-10-06

Features

  • Update type list

Fixes

  • Don't correct typ
  • Consistently error on unused config fields

[1.37.3] - 2025-10-06

... (truncated)

Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=crate-ci/typos&package-manager=github_actions&previous-version=1.39.1&new-version=1.39.2)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/rust.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 64f3a3257101..c58f888df163 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -806,4 +806,4 @@ jobs: - uses: actions/checkout@08c6903cd8c0fde910a37f88322edcfb5dd907a8 # v5.0.0 with: persist-credentials: false - - uses: crate-ci/typos@1af53e3774f068183ffd0c7193eb061a2b65a531 # v1.39.1 + - uses: crate-ci/typos@626c4bedb751ce0b7f03262ca97ddda9a076ae1c # v1.39.2 From ad7bea867566b07209c0c541374f85dc41d32f1c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 14 Nov 2025 10:43:22 +0100 Subject: [PATCH 08/25] Remove FilterExec from CoalesceBatches optimization rule, add fetch support (#18630) 0## Which issue does this PR close? - Closes #18646 ## Rationale for this change Cleans up the plan by removing `CoalesceBatchesExec`. I do not expect any performance improvement. Making plans better to read and avoid useless `CoalesceBatchesExec` Also adds back fetch support by adding it to FilterExec ## What changes are included in this PR? ## Are these changes tested? ## Are there any user-facing changes? --- datafusion/core/tests/dataframe/mod.rs | 113 ++- .../physical_optimizer/limit_pushdown.rs | 7 +- datafusion/core/tests/sql/explain_analyze.rs | 25 +- .../src/coalesce_batches.rs | 9 +- datafusion/physical-plan/src/filter.rs | 93 +- .../sqllogictest/test_files/aggregate.slt | 20 +- datafusion/sqllogictest/test_files/array.slt | 42 +- .../sqllogictest/test_files/async_udf.slt | 11 +- .../test_files/count_star_rule.slt | 13 +- datafusion/sqllogictest/test_files/cte.slt | 39 +- .../sqllogictest/test_files/dictionary.slt | 15 +- .../sqllogictest/test_files/explain.slt | 7 +- .../sqllogictest/test_files/explain_tree.slt | 838 ++++++++---------- .../test_files/filter_without_sort_exec.slt | 42 +- .../sqllogictest/test_files/join.slt.part | 28 +- .../join_disable_repartition_joins.slt | 7 +- .../test_files/join_is_not_distinct_from.slt | 9 +- datafusion/sqllogictest/test_files/joins.slt | 77 +- datafusion/sqllogictest/test_files/limit.slt | 5 +- datafusion/sqllogictest/test_files/map.slt | 5 +- .../sqllogictest/test_files/operator.slt | 40 +- .../sqllogictest/test_files/options.slt | 11 +- .../sqllogictest/test_files/parquet.slt | 28 +- .../test_files/parquet_filter_pushdown.slt | 42 +- .../test_files/parquet_statistics.slt | 21 +- .../sqllogictest/test_files/predicates.slt | 40 +- .../sqllogictest/test_files/projection.slt | 7 +- .../test_files/push_down_filter.slt | 63 +- datafusion/sqllogictest/test_files/pwmj.slt | 46 +- .../sqllogictest/test_files/qualify.slt | 70 +- .../test_files/regexp/regexp_like.slt | 5 +- .../sqllogictest/test_files/repartition.slt | 7 +- .../test_files/repartition_scan.slt | 30 +- datafusion/sqllogictest/test_files/select.slt | 42 +- .../sqllogictest/test_files/simplify_expr.slt | 21 +- .../sqllogictest/test_files/subquery.slt | 32 +- .../test_files/tpch/plans/q1.slt.part | 5 +- .../test_files/tpch/plans/q10.slt.part | 20 +- .../test_files/tpch/plans/q11.slt.part | 50 +- .../test_files/tpch/plans/q12.slt.part | 11 +- .../test_files/tpch/plans/q13.slt.part | 5 +- .../test_files/tpch/plans/q14.slt.part | 11 +- .../test_files/tpch/plans/q15.slt.part | 30 +- .../test_files/tpch/plans/q16.slt.part | 18 +- .../test_files/tpch/plans/q17.slt.part | 19 +- .../test_files/tpch/plans/q18.slt.part | 13 +- .../test_files/tpch/plans/q19.slt.part | 16 +- .../test_files/tpch/plans/q2.slt.part | 99 +-- .../test_files/tpch/plans/q20.slt.part | 55 +- .../test_files/tpch/plans/q21.slt.part | 40 +- .../test_files/tpch/plans/q22.slt.part | 26 +- .../test_files/tpch/plans/q3.slt.part | 25 +- .../test_files/tpch/plans/q4.slt.part | 14 +- .../test_files/tpch/plans/q5.slt.part | 34 +- .../test_files/tpch/plans/q6.slt.part | 5 +- .../test_files/tpch/plans/q7.slt.part | 39 +- .../test_files/tpch/plans/q8.slt.part | 57 +- .../test_files/tpch/plans/q9.slt.part | 37 +- datafusion/sqllogictest/test_files/union.slt | 33 +- datafusion/sqllogictest/test_files/window.slt | 110 +-- 60 files changed, 1205 insertions(+), 1477 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index bb592d967517..cb66fde2976b 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -787,9 +787,8 @@ async fn test_aggregate_with_pk2() -> Result<()> { physical_plan_to_string(&df).await, @r" AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[], ordering_mode=Sorted - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: id@0 = 1 AND name@1 = a - DataSourceExec: partitions=1, partition_sizes=[1] + FilterExec: id@0 = 1 AND name@1 = a + DataSourceExec: partitions=1, partition_sizes=[1] " ); @@ -836,9 +835,8 @@ async fn test_aggregate_with_pk3() -> Result<()> { physical_plan_to_string(&df).await, @r" AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[], ordering_mode=PartiallySorted([0]) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: id@0 = 1 - DataSourceExec: partitions=1, partition_sizes=[1] + FilterExec: id@0 = 1 + DataSourceExec: partitions=1, partition_sizes=[1] " ); @@ -887,9 +885,8 @@ async fn test_aggregate_with_pk4() -> Result<()> { physical_plan_to_string(&df).await, @r" AggregateExec: mode=Single, gby=[id@0 as id], aggr=[], ordering_mode=Sorted - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: id@0 = 1 - DataSourceExec: partitions=1, partition_sizes=[1] + FilterExec: id@0 = 1 + DataSourceExec: partitions=1, partition_sizes=[1] " ); @@ -3375,31 +3372,30 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[a] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | 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] | - | | 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=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ + +---------------+-------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[a] | + | physical_plan | 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] | + | | 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=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-------------------------------------------------------------------------------------------------------------------------+ " ); @@ -3431,31 +3427,30 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t2 projection=[a] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | 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] | - | | 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=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ + +---------------+-------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t2 projection=[a] | + | physical_plan | 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] | + | | 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=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-------------------------------------------------------------------------------------------------------------------------+ " ); diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index 56d48901f284..9d172db246ad 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -199,9 +199,8 @@ fn pushes_global_limit_exec_through_projection_exec() -> Result<()> { let expected = [ "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", - " GlobalLimitExec: skip=0, fetch=5", - " FilterExec: c3@2 > 0", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + " FilterExec: c3@2 > 0, fetch=5", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -310,7 +309,7 @@ fn keeps_pushed_local_limit_exec_when_there_are_multiple_input_partitions() -> R let expected = [ "CoalescePartitionsExec: fetch=5", - " FilterExec: c3@2 > 0", + " FilterExec: c3@2 > 0, fetch=5", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" ]; diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 1a323d07490d..cbc66581e971 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -766,7 +766,7 @@ async fn test_physical_plan_display_indent() { assert_snapshot!( actual, - @r###" + @r" SortPreservingMergeExec: [the_min@2 DESC], fetch=10 SortExec: TopK(fetch=10), expr=[the_min@2 DESC], preserve_partitioning=[true] ProjectionExec: expr=[c1@0 as c1, max(aggregate_test_100.c12)@1 as max(aggregate_test_100.c12), min(aggregate_test_100.c12)@2 as the_min] @@ -774,11 +774,10 @@ async fn test_physical_plan_display_indent() { CoalesceBatchesExec: target_batch_size=4096 RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000 AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)] - 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 - "### + 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 + " ); } @@ -1013,16 +1012,14 @@ async fn parquet_recursive_projection_pushdown() -> Result<()> { RecursiveQueryExec: name=number_series, is_distinct=false CoalescePartitionsExec ProjectionExec: expr=[id@0 as id, 1 as level] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: id@0 = 1 - RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1 - DataSourceExec: file_groups={1 group: [[TMP_DIR/hierarchy.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 = 1, pruning_predicate=id_null_count@2 != row_count@3 AND id_min@0 <= 1 AND 1 <= id_max@1, required_guarantees=[id in (1)] + FilterExec: id@0 = 1 + RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1 + DataSourceExec: file_groups={1 group: [[TMP_DIR/hierarchy.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 = 1, pruning_predicate=id_null_count@2 != row_count@3 AND id_min@0 <= 1 AND 1 <= id_max@1, required_guarantees=[id in (1)] CoalescePartitionsExec ProjectionExec: expr=[id@0 + 1 as ns.id + Int64(1), level@1 + 1 as ns.level + Int64(1)] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: id@0 < 10 - RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1 - WorkTableExec: name=number_series + FilterExec: id@0 < 10 + RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1 + WorkTableExec: name=number_series " ); diff --git a/datafusion/physical-optimizer/src/coalesce_batches.rs b/datafusion/physical-optimizer/src/coalesce_batches.rs index b19d8d9518b3..bde49e71305b 100644 --- a/datafusion/physical-optimizer/src/coalesce_batches.rs +++ b/datafusion/physical-optimizer/src/coalesce_batches.rs @@ -26,7 +26,7 @@ use datafusion_common::error::Result; use datafusion_common::{config::ConfigOptions, internal_err}; use datafusion_physical_expr::Partitioning; use datafusion_physical_plan::{ - async_func::AsyncFuncExec, coalesce_batches::CoalesceBatchesExec, filter::FilterExec, + async_func::AsyncFuncExec, coalesce_batches::CoalesceBatchesExec, joins::HashJoinExec, repartition::RepartitionExec, ExecutionPlan, }; @@ -56,12 +56,7 @@ impl PhysicalOptimizerRule for CoalesceBatches { let target_batch_size = config.execution.batch_size; plan.transform_up(|plan| { let plan_any = plan.as_any(); - // The goal here is to detect operators that could produce small batches and only - // wrap those ones with a CoalesceBatchesExec operator. An alternate approach here - // would be to build the coalescing logic directly into the operators - // See https://github.com/apache/datafusion/issues/139 - let wrap_in_coalesce = plan_any.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() + let wrap_in_coalesce = plan_any.downcast_ref::().is_some() // Don't need to add CoalesceBatchesExec after a round robin RepartitionExec || plan_any .downcast_ref::() diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 0c583e1fb973..8a05bfcee3ab 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -26,6 +26,8 @@ use super::{ ColumnStatistics, DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; +use crate::coalesce::LimitedBatchCoalescer; +use crate::coalesce::PushBatchStatus::LimitReached; use crate::common::can_project; use crate::execution_plan::CardinalityEffect; use crate::filter_pushdown::{ @@ -42,7 +44,7 @@ use crate::{ DisplayFormatType, ExecutionPlan, }; -use arrow::compute::{filter_record_batch, BatchCoalescer}; +use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_boolean_array; @@ -87,6 +89,8 @@ pub struct FilterExec { projection: Option>, /// Target batch size for output batches batch_size: usize, + /// Number of rows to fetch + fetch: Option, } impl FilterExec { @@ -112,6 +116,7 @@ impl FilterExec { cache, projection: None, batch_size: FILTER_EXEC_DEFAULT_BATCH_SIZE, + fetch: None, }) } other => { @@ -160,6 +165,7 @@ impl FilterExec { cache, projection, batch_size: self.batch_size, + fetch: self.fetch, }) } @@ -172,6 +178,7 @@ impl FilterExec { cache: self.cache.clone(), projection: self.projection.clone(), batch_size, + fetch: self.fetch, }) } @@ -351,7 +358,14 @@ impl DisplayAs for FilterExec { } else { "".to_string() }; - write!(f, "FilterExec: {}{}", self.predicate, display_projections) + let fetch = self + .fetch + .map_or_else(|| "".to_string(), |f| format!(", fetch={f}")); + write!( + f, + "FilterExec: {}{}{}", + self.predicate, display_projections, fetch + ) } DisplayFormatType::TreeRender => { write!(f, "predicate={}", fmt_sql(self.predicate.as_ref())) @@ -393,7 +407,7 @@ impl ExecutionPlan for FilterExec { e.with_default_selectivity(selectivity) }) .and_then(|e| e.with_projection(self.projection().cloned())) - .map(|e| Arc::new(e) as _) + .map(|e| e.with_fetch(self.fetch).unwrap()) } fn execute( @@ -409,8 +423,11 @@ impl ExecutionPlan for FilterExec { input: self.input.execute(partition, context)?, metrics, projection: self.projection.clone(), - batch_coalescer: BatchCoalescer::new(self.schema(), self.batch_size) - .with_biggest_coalesce_batch_size(Some(self.batch_size / 2)), + batch_coalescer: LimitedBatchCoalescer::new( + self.schema(), + self.batch_size, + self.fetch, + ), })) } @@ -569,6 +586,7 @@ impl ExecutionPlan for FilterExec { )?, projection: None, batch_size: self.batch_size, + fetch: self.fetch, }; Some(Arc::new(new) as _) }; @@ -578,6 +596,19 @@ impl ExecutionPlan for FilterExec { updated_node, }) } + + fn with_fetch(&self, fetch: Option) -> Option> { + Some(Arc::new(Self { + predicate: Arc::clone(&self.predicate), + input: Arc::clone(&self.input), + metrics: self.metrics.clone(), + default_selectivity: self.default_selectivity, + cache: self.cache.clone(), + projection: self.projection.clone(), + batch_size: self.batch_size, + fetch, + })) + } } impl EmbeddedProjection for FilterExec { @@ -648,7 +679,7 @@ struct FilterExecStream { /// The projection indices of the columns in the input schema projection: Option>, /// Batch coalescer to combine small batches - batch_coalescer: BatchCoalescer, + batch_coalescer: LimitedBatchCoalescer, } /// The metrics for `FilterExec` @@ -670,6 +701,23 @@ impl FilterExecMetrics { } } +impl FilterExecStream { + fn flush_remaining_batches( + &mut self, + ) -> Poll>> { + // Flush any remaining buffered batch + match self.batch_coalescer.finish() { + Ok(()) => { + Poll::Ready(self.batch_coalescer.next_completed_batch().map(|batch| { + self.metrics.selectivity.add_part(batch.num_rows()); + Ok(batch) + })) + } + Err(e) => Poll::Ready(Some(Err(e))), + } + } +} + pub fn batch_filter( batch: &RecordBatch, predicate: &Arc, @@ -715,7 +763,7 @@ impl Stream for FilterExecStream { match ready!(self.input.poll_next_unpin(cx)) { Some(Ok(batch)) => { let timer = elapsed_compute.timer(); - self.predicate.as_ref() + let status = self.predicate.as_ref() .evaluate(&batch) .and_then(|v| v.into_array(batch.num_rows())) .and_then(|array| { @@ -729,11 +777,11 @@ impl Stream for FilterExecStream { }).and_then(|(array, batch)| { match as_boolean_array(&array) { Ok(filter_array) => { - self.metrics.selectivity.add_part(filter_array.true_count()); self.metrics.selectivity.add_total(batch.num_rows()); - - self.batch_coalescer.push_batch_with_filter(batch.clone(), filter_array)?; - Ok(()) + // TODO: support push_batch_with_filter in LimitedBatchCoalescer + let batch = filter_record_batch(&batch, filter_array)?; + let state = self.batch_coalescer.push_batch(batch)?; + Ok(state) } Err(_) => { internal_err!( @@ -742,28 +790,28 @@ impl Stream for FilterExecStream { } } })?; - timer.done(); - if self.batch_coalescer.has_completed_batch() { - poll = Poll::Ready(Some(Ok(self - .batch_coalescer - .next_completed_batch() - .expect("has_completed_batch is true")))); + if let LimitReached = status { + poll = self.flush_remaining_batches(); + break; + } + + if let Some(batch) = self.batch_coalescer.next_completed_batch() { + self.metrics.selectivity.add_part(batch.num_rows()); + poll = Poll::Ready(Some(Ok(batch))); break; } continue; } None => { // Flush any remaining buffered batch - match self.batch_coalescer.finish_buffered_batch() { + match self.batch_coalescer.finish() { Ok(()) => { - poll = Poll::Ready( - self.batch_coalescer.next_completed_batch().map(Ok), - ); + poll = self.flush_remaining_batches(); } Err(e) => { - poll = Poll::Ready(Some(Err(e.into()))); + poll = Poll::Ready(Some(Err(e))); } } break; @@ -782,7 +830,6 @@ impl Stream for FilterExecStream { self.input.size_hint() } } - impl RecordBatchStream for FilterExecStream { fn schema(&self) -> SchemaRef { Arc::clone(&self.schema) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 0445df90f682..8dce114ab55f 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -6045,10 +6045,9 @@ physical_plan 07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 08)--------------CoalescePartitionsExec 09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c2, c3], file_type=csv, has_header=true +10)------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.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; @@ -7193,13 +7192,12 @@ logical_plan 03)----Aggregate: groupBy=[[having_test.v1, having_test.v2]], aggr=[[max(having_test.v1)]] 04)------TableScan: having_test projection=[v1, v2] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] -03)----AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] +02)--AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=1 +05)--------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query error diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 7aa267a4dc6d..c69e7a19e4f7 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -6436,10 +6436,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------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)) @@ -6465,10 +6464,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------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)) @@ -6494,10 +6492,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------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)) @@ -6523,10 +6520,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------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)) @@ -6552,10 +6548,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------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)) @@ -6583,10 +6578,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 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] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IS NOT NULL OR NULL +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # any operator query ? diff --git a/datafusion/sqllogictest/test_files/async_udf.slt b/datafusion/sqllogictest/test_files/async_udf.slt index c61d02cfecfd..9bca72a2fdfb 100644 --- a/datafusion/sqllogictest/test_files/async_udf.slt +++ b/datafusion/sqllogictest/test_files/async_udf.slt @@ -80,12 +80,11 @@ logical_plan 01)Filter: async_abs(data.x) < Int32(5) 02)--TableScan: data projection=[x] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: __async_fn_0@1 < 5, projection=[x@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------AsyncFuncExec: async_expr=[async_expr(name=__async_fn_0, expr=async_abs(x@0))] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: __async_fn_0@1 < 5, projection=[x@0] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----AsyncFuncExec: async_expr=[async_expr(name=__async_fn_0, expr=async_abs(x@0))] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] # Async udf can be used in projection query I rowsort diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index 19d9ddecc9ff..38676baddd99 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -64,13 +64,12 @@ logical_plan 04)------TableScan: t1 projection=[a] physical_plan 01)ProjectionExec: expr=[a@0 as a, count(Int64(1))@1 as cnt] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: count(Int64(1))@1 > 0 -04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: count(Int64(1))@1 > 0 +03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 1; diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index e7ca7a5ae1d8..03900a608e6a 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -117,10 +117,9 @@ physical_plan 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 +06)------FilterExec: id@0 < 10 +07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------WorkTableExec: name=nodes # setup statement ok @@ -162,10 +161,9 @@ physical_plan 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 +06)--------FilterExec: time@0 < 10 +07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------------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 @@ -734,12 +732,11 @@ physical_plan 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 +07)--------FilterExec: val@0 < 2 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------WorkTableExec: name=recursive_cte +10)------ProjectionExec: expr=[2 as val] +11)--------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/9794 # Non-recursive term and recursive term have different types @@ -964,10 +961,9 @@ physical_plan 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 +06)------FilterExec: n@0 < 10 +07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------WorkTableExec: name=numbers query TT explain WITH RECURSIVE numbers AS ( @@ -990,10 +986,9 @@ physical_plan 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 +06)------FilterExec: n@0 < 10 +07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------WorkTableExec: name=numbers # Test for issue #16998: SortExec shares DynamicFilterPhysicalExpr across multiple executions query II diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt index fd9a7fb9ce44..b6098758a9e6 100644 --- a/datafusion/sqllogictest/test_files/dictionary.slt +++ b/datafusion/sqllogictest/test_files/dictionary.slt @@ -410,9 +410,8 @@ logical_plan 01)Filter: test.column2 = Dictionary(Int32, Utf8("1")) 02)--TableScan: test projection=[column1, column2] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column2@1 = 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: column2@1 = 1 +02)--DataSourceExec: partitions=1, partition_sizes=[1] # try literal = col to verify order doesn't matter # filter should not cast column2 @@ -423,9 +422,8 @@ logical_plan 01)Filter: test.column2 = Dictionary(Int32, Utf8("1")) 02)--TableScan: test projection=[column1, column2] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column2@1 = 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: column2@1 = 1 +02)--DataSourceExec: partitions=1, partition_sizes=[1] # Now query using an integer which must be coerced into a dictionary string @@ -441,9 +439,8 @@ logical_plan 01)Filter: test.column2 = Dictionary(Int32, Utf8("1")) 02)--TableScan: test projection=[column1, column2] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column2@1 = 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: column2@1 = 1 +02)--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 ec3d9f746577..d7af5ff4b9c2 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -43,10 +43,9 @@ logical_plan 02)--Filter: aggregate_test_100.c2 > Int8(10) 03)----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan -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 +01)FilterExec: c2@1 > 10, projection=[c1@0] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----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 diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 5f3c778fc961..6ac28997a990 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -166,32 +166,26 @@ explain SELECT int_col FROM table1 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: csv │ +21)└───────────────────────────┘ # Aggregate query TT @@ -411,36 +405,30 @@ WHERE string_col != 'foo' AND string_col != 'bar' AND string_col != 'a really lo ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ FilterExec │ -09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo AND │ -12)│ string_col != bar │ -13)│ AND string_col != a │ -14)│ really long string │ -15)│ constant │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ RepartitionExec │ -19)│ -------------------- │ -20)│ partition_count(in->out): │ -21)│ 1 -> 4 │ -22)│ │ -23)│ partitioning_scheme: │ -24)│ RoundRobinBatch(4) │ -25)└─────────────┬─────────────┘ -26)┌─────────────┴─────────────┐ -27)│ DataSourceExec │ -28)│ -------------------- │ -29)│ files: 1 │ -30)│ format: csv │ -31)└───────────────────────────┘ +04)│ predicate: │ +05)│ string_col != foo AND │ +06)│ string_col != bar │ +07)│ AND string_col != a │ +08)│ really long string │ +09)│ constant │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)│ -------------------- │ +14)│ partition_count(in->out): │ +15)│ 1 -> 4 │ +16)│ │ +17)│ partitioning_scheme: │ +18)│ RoundRobinBatch(4) │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ DataSourceExec │ +22)│ -------------------- │ +23)│ files: 1 │ +24)│ format: csv │ +25)└───────────────────────────┘ # Check maximum line limit. query TT @@ -449,17 +437,17 @@ WHERE string_col != 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ FilterExec │ -09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != │ -12)│ aaaaaaaaaaaa │ +04)│ predicate: │ +05)│ string_col != │ +06)│ aaaaaaaaaaaa │ +07)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +08)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +09)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +10)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +11)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +12)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 13)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 14)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 15)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ @@ -480,29 +468,23 @@ physical_plan 30)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 31)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 32)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -33)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -34)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -35)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -36)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -37)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -38)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -39)│ ... │ -40)└─────────────┬─────────────┘ -41)┌─────────────┴─────────────┐ -42)│ RepartitionExec │ -43)│ -------------------- │ -44)│ partition_count(in->out): │ -45)│ 1 -> 4 │ -46)│ │ -47)│ partitioning_scheme: │ -48)│ RoundRobinBatch(4) │ -49)└─────────────┬─────────────┘ -50)┌─────────────┴─────────────┐ -51)│ DataSourceExec │ -52)│ -------------------- │ -53)│ files: 1 │ -54)│ format: csv │ -55)└───────────────────────────┘ +33)│ ... │ +34)└─────────────┬─────────────┘ +35)┌─────────────┴─────────────┐ +36)│ RepartitionExec │ +37)│ -------------------- │ +38)│ partition_count(in->out): │ +39)│ 1 -> 4 │ +40)│ │ +41)│ partitioning_scheme: │ +42)│ RoundRobinBatch(4) │ +43)└─────────────┬─────────────┘ +44)┌─────────────┴─────────────┐ +45)│ DataSourceExec │ +46)│ -------------------- │ +47)│ files: 1 │ +48)│ format: csv │ +49)└───────────────────────────┘ # Check exactly the render width. query TT @@ -511,32 +493,26 @@ WHERE string_col != 'aaaaaaaaaaaaa'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│string_col != aaaaaaaaaaaaa│ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│string_col != aaaaaaaaaaaaa│ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: csv │ +21)└───────────────────────────┘ # Check with the render witdth + 1. query TT @@ -545,34 +521,28 @@ WHERE string_col != 'aaaaaaaaaaaaaaa'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ FilterExec │ -09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != │ -12)│ aaaaaaaaaaaa │ -13)│ aaa │ -14)└─────────────┬─────────────┘ -15)┌─────────────┴─────────────┐ -16)│ RepartitionExec │ -17)│ -------------------- │ -18)│ partition_count(in->out): │ -19)│ 1 -> 4 │ -20)│ │ -21)│ partitioning_scheme: │ -22)│ RoundRobinBatch(4) │ -23)└─────────────┬─────────────┘ -24)┌─────────────┴─────────────┐ -25)│ DataSourceExec │ -26)│ -------------------- │ -27)│ files: 1 │ -28)│ format: csv │ -29)└───────────────────────────┘ +04)│ predicate: │ +05)│ string_col != │ +06)│ aaaaaaaaaaaa │ +07)│ aaa │ +08)└─────────────┬─────────────┘ +09)┌─────────────┴─────────────┐ +10)│ RepartitionExec │ +11)│ -------------------- │ +12)│ partition_count(in->out): │ +13)│ 1 -> 4 │ +14)│ │ +15)│ partitioning_scheme: │ +16)│ RoundRobinBatch(4) │ +17)└─────────────┬─────────────┘ +18)┌─────────────┴─────────────┐ +19)│ DataSourceExec │ +20)│ -------------------- │ +21)│ files: 1 │ +22)│ format: csv │ +23)└───────────────────────────┘ # Query with filter on csv query TT @@ -580,32 +550,26 @@ explain SELECT int_col FROM table1 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: csv │ +21)└───────────────────────────┘ # Query with filter on parquet @@ -614,35 +578,29 @@ explain SELECT int_col FROM table2 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: parquet │ -27)│ │ -28)│ predicate: │ -29)│ string_col != foo │ -30)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: parquet │ +21)│ │ +22)│ predicate: │ +23)│ string_col != foo │ +24)└───────────────────────────┘ # Query with filter on memory query TT @@ -650,24 +608,18 @@ explain SELECT int_col FROM table3 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ DataSourceExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ DataSourceExec │ -15)│ -------------------- │ -16)│ bytes: 520 │ -17)│ format: memory │ -18)│ rows: 1 │ -19)└───────────────────────────┘ +10)│ bytes: 520 │ +11)│ format: memory │ +12)│ rows: 1 │ +13)└───────────────────────────┘ # Query with filter on json query TT @@ -675,32 +627,26 @@ explain SELECT int_col FROM table4 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: json │ -27)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: json │ +21)└───────────────────────────┘ # Query with filter on arrow query TT @@ -708,32 +654,26 @@ explain SELECT int_col FROM table5 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: arrow │ -27)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: arrow │ +21)└───────────────────────────┘ # Query with window agg. @@ -1483,31 +1423,25 @@ physical_plan 05)│ ASC NULLS LAST │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ CoalesceBatchesExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ target_batch_size: │ -11)│ 8192 │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ FilterExec │ -15)│ -------------------- │ -16)│ predicate: ticker = A │ -17)└─────────────┬─────────────┘ -18)┌─────────────┴─────────────┐ -19)│ RepartitionExec │ -20)│ -------------------- │ -21)│ partition_count(in->out): │ -22)│ 1 -> 4 │ -23)│ │ -24)│ partitioning_scheme: │ -25)│ RoundRobinBatch(4) │ -26)└─────────────┬─────────────┘ -27)┌─────────────┴─────────────┐ -28)│ StreamingTableExec │ -29)│ -------------------- │ -30)│ infinite: true │ -31)│ limit: None │ -32)└───────────────────────────┘ +10)│ predicate: ticker = A │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ RepartitionExec │ +14)│ -------------------- │ +15)│ partition_count(in->out): │ +16)│ 1 -> 4 │ +17)│ │ +18)│ partitioning_scheme: │ +19)│ RoundRobinBatch(4) │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ StreamingTableExec │ +23)│ -------------------- │ +24)│ infinite: true │ +25)│ limit: None │ +26)└───────────────────────────┘ # constant ticker, CAST(time AS DATE) = time, order by time @@ -1523,33 +1457,27 @@ physical_plan 04)│ time ASC NULLS LAST │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ CoalesceBatchesExec │ +07)│ FilterExec │ 08)│ -------------------- │ -09)│ target_batch_size: │ -10)│ 8192 │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ FilterExec │ -14)│ -------------------- │ -15)│ predicate: │ -16)│ ticker = A AND CAST(time │ -17)│ AS Date32) = date │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ partition_count(in->out): │ -23)│ 1 -> 4 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ -30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +09)│ predicate: │ +10)│ ticker = A AND CAST(time │ +11)│ AS Date32) = date │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ StreamingTableExec │ +24)│ -------------------- │ +25)│ infinite: true │ +26)│ limit: None │ +27)└───────────────────────────┘ # same thing but order by date query TT @@ -1564,33 +1492,27 @@ physical_plan 04)│ date ASC NULLS LAST │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ CoalesceBatchesExec │ +07)│ FilterExec │ 08)│ -------------------- │ -09)│ target_batch_size: │ -10)│ 8192 │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ FilterExec │ -14)│ -------------------- │ -15)│ predicate: │ -16)│ ticker = A AND CAST(time │ -17)│ AS Date32) = date │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ partition_count(in->out): │ -23)│ 1 -> 4 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ -30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +09)│ predicate: │ +10)│ ticker = A AND CAST(time │ +11)│ AS Date32) = date │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ StreamingTableExec │ +24)│ -------------------- │ +25)│ infinite: true │ +26)│ limit: None │ +27)└───────────────────────────┘ # same thing but order by ticker query TT @@ -1603,33 +1525,27 @@ physical_plan 02)│ CoalescePartitionsExec │ 03)└─────────────┬─────────────┘ 04)┌─────────────┴─────────────┐ -05)│ CoalesceBatchesExec │ +05)│ FilterExec │ 06)│ -------------------- │ -07)│ target_batch_size: │ -08)│ 8192 │ -09)└─────────────┬─────────────┘ -10)┌─────────────┴─────────────┐ -11)│ FilterExec │ -12)│ -------------------- │ -13)│ predicate: │ -14)│ ticker = A AND CAST(time │ -15)│ AS Date32) = date │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ RepartitionExec │ -19)│ -------------------- │ -20)│ partition_count(in->out): │ -21)│ 1 -> 4 │ -22)│ │ -23)│ partitioning_scheme: │ -24)│ RoundRobinBatch(4) │ -25)└─────────────┬─────────────┘ -26)┌─────────────┴─────────────┐ -27)│ StreamingTableExec │ -28)│ -------------------- │ -29)│ infinite: true │ -30)│ limit: None │ -31)└───────────────────────────┘ +07)│ predicate: │ +08)│ ticker = A AND CAST(time │ +09)│ AS Date32) = date │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)│ -------------------- │ +14)│ partition_count(in->out): │ +15)│ 1 -> 4 │ +16)│ │ +17)│ partitioning_scheme: │ +18)│ RoundRobinBatch(4) │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ StreamingTableExec │ +22)│ -------------------- │ +23)│ infinite: true │ +24)│ limit: None │ +25)└───────────────────────────┘ # same thing but order by time, date @@ -1646,33 +1562,27 @@ physical_plan 05)│ ASC NULLS LAST │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ CoalesceBatchesExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ target_batch_size: │ -11)│ 8192 │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ FilterExec │ -15)│ -------------------- │ -16)│ predicate: │ -17)│ ticker = A AND CAST(time │ -18)│ AS Date32) = date │ -19)└─────────────┬─────────────┘ -20)┌─────────────┴─────────────┐ -21)│ RepartitionExec │ -22)│ -------------------- │ -23)│ partition_count(in->out): │ -24)│ 1 -> 4 │ -25)│ │ -26)│ partitioning_scheme: │ -27)│ RoundRobinBatch(4) │ -28)└─────────────┬─────────────┘ -29)┌─────────────┴─────────────┐ -30)│ StreamingTableExec │ -31)│ -------------------- │ -32)│ infinite: true │ -33)│ limit: None │ -34)└───────────────────────────┘ +10)│ predicate: │ +11)│ ticker = A AND CAST(time │ +12)│ AS Date32) = date │ +13)└─────────────┬─────────────┘ +14)┌─────────────┴─────────────┐ +15)│ RepartitionExec │ +16)│ -------------------- │ +17)│ partition_count(in->out): │ +18)│ 1 -> 4 │ +19)│ │ +20)│ partitioning_scheme: │ +21)│ RoundRobinBatch(4) │ +22)└─────────────┬─────────────┘ +23)┌─────────────┴─────────────┐ +24)│ StreamingTableExec │ +25)│ -------------------- │ +26)│ infinite: true │ +27)│ limit: None │ +28)└───────────────────────────┘ @@ -1693,34 +1603,26 @@ physical_plan 07)│ time ASC NULLS LAST │ 08)└─────────────┬─────────────┘ 09)┌─────────────┴─────────────┐ -10)│ CoalesceBatchesExec │ +10)│ FilterExec │ 11)│ -------------------- │ -12)│ limit: 5 │ -13)│ │ -14)│ target_batch_size: │ -15)│ 8192 │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ FilterExec │ -19)│ -------------------- │ -20)│ predicate: │ -21)│ date = 2006-01-02 │ -22)└─────────────┬─────────────┘ -23)┌─────────────┴─────────────┐ -24)│ RepartitionExec │ -25)│ -------------------- │ -26)│ partition_count(in->out): │ -27)│ 1 -> 4 │ -28)│ │ -29)│ partitioning_scheme: │ -30)│ RoundRobinBatch(4) │ -31)└─────────────┬─────────────┘ -32)┌─────────────┴─────────────┐ -33)│ StreamingTableExec │ -34)│ -------------------- │ -35)│ infinite: true │ -36)│ limit: None │ -37)└───────────────────────────┘ +12)│ predicate: │ +13)│ date = 2006-01-02 │ +14)└─────────────┬─────────────┘ +15)┌─────────────┴─────────────┐ +16)│ RepartitionExec │ +17)│ -------------------- │ +18)│ partition_count(in->out): │ +19)│ 1 -> 4 │ +20)│ │ +21)│ partitioning_scheme: │ +22)│ RoundRobinBatch(4) │ +23)└─────────────┬─────────────┘ +24)┌─────────────┴─────────────┐ +25)│ StreamingTableExec │ +26)│ -------------------- │ +27)│ infinite: true │ +28)│ limit: None │ +29)└───────────────────────────┘ @@ -1739,32 +1641,26 @@ physical_plan 05)│ time ASC NULLS LAST │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ CoalesceBatchesExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ target_batch_size: │ -11)│ 8192 │ +10)│ predicate: │ +11)│ date = 2006-01-02 │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ FilterExec │ +14)│ RepartitionExec │ 15)│ -------------------- │ -16)│ predicate: │ -17)│ date = 2006-01-02 │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ partition_count(in->out): │ -23)│ 1 -> 4 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ -30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ StreamingTableExec │ +24)│ -------------------- │ +25)│ infinite: true │ +26)│ limit: None │ +27)└───────────────────────────┘ @@ -1794,30 +1690,24 @@ physical_plan 12)│ ││ id: id + 1 │ 13)└───────────────────────────┘└─────────────┬─────────────┘ 14)-----------------------------┌─────────────┴─────────────┐ -15)-----------------------------│ CoalesceBatchesExec │ +15)-----------------------------│ FilterExec │ 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)-----------------------------└───────────────────────────┘ +17)-----------------------------│ predicate: id < 10 │ +18)-----------------------------└─────────────┬─────────────┘ +19)-----------------------------┌─────────────┴─────────────┐ +20)-----------------------------│ RepartitionExec │ +21)-----------------------------│ -------------------- │ +22)-----------------------------│ partition_count(in->out): │ +23)-----------------------------│ 1 -> 4 │ +24)-----------------------------│ │ +25)-----------------------------│ partitioning_scheme: │ +26)-----------------------------│ RoundRobinBatch(4) │ +27)-----------------------------└─────────────┬─────────────┘ +28)-----------------------------┌─────────────┴─────────────┐ +29)-----------------------------│ WorkTableExec │ +30)-----------------------------│ -------------------- │ +31)-----------------------------│ name: nodes │ +32)-----------------------------└───────────────────────────┘ query TT explain COPY (VALUES (1, 'foo', 1, '2023-01-01'), (2, 'bar', 2, '2023-01-02'), (3, 'baz', 3, '2023-01-03')) @@ -1937,25 +1827,17 @@ physical_plan 38)│ skip: 6 │ 39)└─────────────┬─────────────┘ 40)┌─────────────┴─────────────┐ -41)│ CoalesceBatchesExec │ +41)│ FilterExec │ 42)│ -------------------- │ -43)│ limit: 9 │ -44)│ │ -45)│ target_batch_size: │ -46)│ 8192 │ -47)└─────────────┬─────────────┘ -48)┌─────────────┴─────────────┐ -49)│ FilterExec │ -50)│ -------------------- │ -51)│ predicate: a > 3 │ -52)└─────────────┬─────────────┘ -53)┌─────────────┴─────────────┐ -54)│ DataSourceExec │ -55)│ -------------------- │ -56)│ bytes: 160 │ -57)│ format: memory │ -58)│ rows: 1 │ -59)└───────────────────────────┘ +43)│ predicate: a > 3 │ +44)└─────────────┬─────────────┘ +45)┌─────────────┴─────────────┐ +46)│ DataSourceExec │ +47)│ -------------------- │ +48)│ bytes: 160 │ +49)│ format: memory │ +50)│ rows: 1 │ +51)└───────────────────────────┘ # clean up statement ok @@ -1985,33 +1867,25 @@ physical_plan 04)│ limit: 5 │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ CoalesceBatchesExec │ +07)│ FilterExec │ 08)│ -------------------- │ -09)│ limit: 5 │ -10)│ │ -11)│ target_batch_size: │ -12)│ 8192 │ -13)└─────────────┬─────────────┘ -14)┌─────────────┴─────────────┐ -15)│ FilterExec │ -16)│ -------------------- │ -17)│ predicate: c3 > 0 │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ partition_count(in->out): │ -23)│ 1 -> 4 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ -30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +09)│ predicate: c3 > 0 │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)│ -------------------- │ +14)│ partition_count(in->out): │ +15)│ 1 -> 4 │ +16)│ │ +17)│ partitioning_scheme: │ +18)│ RoundRobinBatch(4) │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ StreamingTableExec │ +22)│ -------------------- │ +23)│ infinite: true │ +24)│ limit: None │ +25)└───────────────────────────┘ # Test explain tree for PlaceholderRowExec query TT diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index a09d8ce26ddf..be79f1423859 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -38,10 +38,9 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST, time@2 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: ticker@1 = A +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------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 @@ -55,10 +54,9 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------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 @@ -72,10 +70,9 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------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 @@ -89,10 +86,9 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)CoalescePartitionsExec -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------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 @@ -106,10 +102,9 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST, date@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------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 @@ -147,7 +142,6 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] -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] +02)--FilterExec: date@0 = 2006-01-02 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------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/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 87373af1472a..ea998f742d30 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -936,10 +936,9 @@ logical_plan 06)----TableScan: department projection=[dept_name] 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] +02)--FilterExec: name@1 = Alice OR name@1 = Bob +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)--DataSourceExec: partitions=1, partition_sizes=[1] # expect no row for Carol query ITT @@ -984,15 +983,13 @@ logical_plan 07)------SubqueryAlias: d 08)--------TableScan: department projection=[emp_id, dept_name] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CoalesceBatchesExec: target_batch_size=8192 -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] +01)FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] +05)--------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------DataSourceExec: partitions=1, partition_sizes=[1] query ITT SELECT e.emp_id, e.name, d.dept_name @@ -1386,9 +1383,8 @@ physical_plan 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] +13)----------------FilterExec: y@1 = x@0 +14)------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table pairs; diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index a1efc1317b4a..e78169734fe5 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -99,10 +99,9 @@ physical_plan 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 +06)--------FilterExec: d@3 = 3 +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, 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/join_is_not_distinct_from.slt b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt index 0336cfc2d331..166d1a028ecb 100644 --- a/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt +++ b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt @@ -153,11 +153,10 @@ physical_plan 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t1.val + Int64(1)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------FilterExec: CAST(val@1 AS Int64) + 1 IS NOT DISTINCT FROM 11 -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] -11)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------FilterExec: CAST(val@1 AS Int64) + 1 IS NOT DISTINCT FROM 11 +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] +10)--------DataSourceExec: partitions=1, partition_sizes=[1] # Mixed join predicate with `IS DISTINCT FROM` and `IS NOT DISTINCT FROM` query IIII rowsort diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 62804ad76bd6..4a243258a519 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2052,14 +2052,12 @@ physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id] 02)--NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 03)----CoalescePartitionsExec -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] +04)------FilterExec: t2_int@1 > 1, projection=[t2_id@0] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----FilterExec: t1_id@0 > 10 +08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2092,14 +2090,12 @@ logical_plan physical_plan 01)NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1 02)--CoalescePartitionsExec -03)----CoalesceBatchesExec: target_batch_size=2 -04)------FilterExec: t1_id@0 > 22 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------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] +03)----FilterExec: t1_id@0 > 22 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--FilterExec: t2_id@0 > 11 +07)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -4108,9 +4104,8 @@ physical_plan 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] +07)----------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] +08)------------DataSourceExec: partitions=1, partition_sizes=[0] statement ok DROP TABLE sales_global; @@ -4459,11 +4454,10 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalescePartitionsExec -05)--------CoalesceBatchesExec: target_batch_size=3 -06)----------FilterExec: b@1 > 3, projection=[a@0] -07)------------DataSourceExec: partitions=2, partition_sizes=[1, 1] -08)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] -09)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] +05)--------FilterExec: b@1 > 3, projection=[a@0] +06)----------DataSourceExec: partitions=2, partition_sizes=[1, 1] +07)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] +08)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] query TT explain select * from test where a in (select a from test where b > 3) order by c desc nulls last; @@ -4481,11 +4475,10 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalescePartitionsExec -05)--------CoalesceBatchesExec: target_batch_size=3 -06)----------FilterExec: b@1 > 3, projection=[a@0] -07)------------DataSourceExec: partitions=2, partition_sizes=[1, 1] -08)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] -09)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] +05)--------FilterExec: b@1 > 3, projection=[a@0] +06)----------DataSourceExec: partitions=2, partition_sizes=[1, 1] +07)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] +08)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] query III select * from test where a in (select a from test where b > 3) order by c desc nulls first; @@ -5085,12 +5078,11 @@ WHERE k2 > 0 ) ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--FilterExec: k2@0 > 0 OR mark@2, projection=[k2@0, v2@1] -03)----CoalesceBatchesExec: target_batch_size=3 -04)------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(k2@0, k1@0)] -05)--------DataSourceExec: partitions=1, partition_sizes=[0] -06)--------DataSourceExec: partitions=1, partition_sizes=[10000] +01)FilterExec: k2@0 > 0 OR mark@2, projection=[k2@0, v2@1] +02)--CoalesceBatchesExec: target_batch_size=3 +03)----HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(k2@0, k1@0)] +04)------DataSourceExec: partitions=1, partition_sizes=[0] +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT * @@ -5136,9 +5128,8 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----CoalesceBatchesExec: target_batch_size=3 -05)------FilterExec: k1@0 < 0 -06)--------DataSourceExec: partitions=1, partition_sizes=[10000] +04)----FilterExec: k1@0 < 0 +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT * @@ -5205,12 +5196,10 @@ physical_plan 01)SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--PiecewiseMergeJoin: operator=Gt, join_type=Inner, on=(t1_id > t2_id) 03)----SortExec: expr=[t1_id@0 ASC], preserve_partitioning=[false] -04)------CoalesceBatchesExec: target_batch_size=3 -05)--------FilterExec: t1_id@0 > 10 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----CoalesceBatchesExec: target_batch_size=3 -08)------FilterExec: t2_int@1 > 1, projection=[t2_id@0] -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------FilterExec: t1_id@0 > 10 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)----FilterExec: t2_int@1 > 1, projection=[t2_id@0] +07)------DataSourceExec: partitions=1, partition_sizes=[1] statement ok DROP TABLE t1; diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 25b741b025a7..48781e46f11c 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -377,9 +377,8 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------ProjectionExec: expr=[] 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] +08)--------------FilterExec: a@0 > 3, fetch=9 +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query I SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 45f8c5d25fbe..75acd68432c1 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -113,9 +113,8 @@ logical_plan 01)Filter: table_with_map.int_field > Int64(0) 02)--TableScan: table_with_map projection=[int_field, map_field] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: int_field@0 > 0 -03)----DataSourceExec: partitions=1, partition_sizes=[0] +01)FilterExec: int_field@0 > 0 +02)--DataSourceExec: partitions=1, partition_sizes=[0] statement ok drop table table_with_map; diff --git a/datafusion/sqllogictest/test_files/operator.slt b/datafusion/sqllogictest/test_files/operator.slt index 6f3c40188172..e50fa721c885 100644 --- a/datafusion/sqllogictest/test_files/operator.slt +++ b/datafusion/sqllogictest/test_files/operator.slt @@ -287,9 +287,8 @@ EXPLAIN SELECT * FROM numeric_types 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] +01)FilterExec: int64@3 < 5 AND uint64@7 < 5 AND float64@9 < 5 AND decimal@10 < Some(500),5,2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## < negative integer (expect no casts) query TT @@ -297,9 +296,8 @@ EXPLAIN SELECT * FROM numeric_types 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] +01)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 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## < decimal (expect casts for integers to float) query TT @@ -307,9 +305,8 @@ EXPLAIN SELECT * FROM numeric_types 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] +01)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 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## < negative decimal (expect casts for integers to float) query TT @@ -317,9 +314,8 @@ EXPLAIN SELECT * FROM numeric_types 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] +01)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 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ############### Equality ############### @@ -330,9 +326,8 @@ EXPLAIN SELECT * FROM numeric_types 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] +01)FilterExec: int64@3 = 5 AND uint64@7 = 5 AND float64@9 = 5 AND decimal@10 = Some(500),5,2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## = negative integer (expect no casts) query TT @@ -340,9 +335,8 @@ EXPLAIN SELECT * FROM numeric_types 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] +01)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 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## = decimal (expect casts for integers to float) query TT @@ -350,9 +344,8 @@ EXPLAIN SELECT * FROM numeric_types 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] +01)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 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## = negative decimal (expect casts for integers to float) query TT @@ -360,9 +353,8 @@ EXPLAIN SELECT * FROM numeric_types 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] +01)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 +02)--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..0d1583dbc008 100644 --- a/datafusion/sqllogictest/test_files/options.slt +++ b/datafusion/sqllogictest/test_files/options.slt @@ -23,7 +23,6 @@ statement ok create table a(c0 int) as values (1), (2); -# Expect coalesce and default batch size query TT explain SELECT * FROM a WHERE c0 < 1; ---- @@ -31,9 +30,8 @@ logical_plan 01)Filter: a.c0 < Int32(1) 02)--TableScan: a projection=[c0] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: c0@0 < 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: c0@0 < 1 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## # test_disable_coalesce @@ -72,9 +70,8 @@ logical_plan 01)Filter: a.c0 < Int32(1) 02)--TableScan: a projection=[c0] physical_plan -01)CoalesceBatchesExec: target_batch_size=1234 -02)--FilterExec: c0@0 < 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: c0@0 < 1 +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index c21f3129d4ee..c786f7bdc77c 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -454,10 +454,9 @@ logical_plan 01)Filter: CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%") AND CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%") AND CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%") 02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%")] 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% +01)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% +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----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 @@ -502,10 +501,9 @@ logical_plan 01)Filter: binary_as_string_option.binary_col LIKE Utf8View("%a%") AND binary_as_string_option.largebinary_col LIKE Utf8View("%a%") AND binary_as_string_option.binaryview_col LIKE Utf8View("%a%") 02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8View("%a%"), binary_as_string_option.largebinary_col LIKE Utf8View("%a%"), binary_as_string_option.binaryview_col LIKE Utf8View("%a%")] 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% +01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----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 @@ -553,10 +551,9 @@ logical_plan 01)Filter: binary_as_string_both.binary_col LIKE Utf8View("%a%") AND binary_as_string_both.largebinary_col LIKE Utf8View("%a%") AND binary_as_string_both.binaryview_col LIKE Utf8View("%a%") 02)--TableScan: binary_as_string_both projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_both.binary_col LIKE Utf8View("%a%"), binary_as_string_both.largebinary_col LIKE Utf8View("%a%"), binary_as_string_both.binaryview_col LIKE Utf8View("%a%")] 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% +01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----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 @@ -668,10 +665,9 @@ logical_plan 01)Filter: foo.column1 LIKE Utf8View("f%") 02)--TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8View("f%")] 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=[] +01)FilterExec: column1@0 LIKE f% +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----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 0166cd2572ce..8bb79d576990 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -95,10 +95,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -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=[] +03)----FilterExec: b@1 > 2, projection=[a@0] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +05)--------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=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -133,11 +132,9 @@ logical_plan 04)------TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--ProjectionExec: expr=[a@0 as a] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: b@1 = 2 -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@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--FilterExec: b@1 = 2, projection=[a@0] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +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, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -266,10 +263,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -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=[] +03)----FilterExec: b@1 > 2, projection=[a@0] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +05)--------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=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -304,11 +300,9 @@ logical_plan 04)------TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--ProjectionExec: expr=[a@0 as a] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: b@1 = 2 -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@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--FilterExec: b@1 = 2, projection=[a@0] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +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, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -344,10 +338,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -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=[] +03)----FilterExec: b@1 > 2, projection=[a@0] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +05)--------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=[] query T select a from t_pushdown where b = 2 ORDER BY b; @@ -416,9 +409,8 @@ logical_plan 02)--Filter: CAST(t_pushdown.b AS Float64) > random() 03)----TableScan: t_pushdown projection=[a, b] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: CAST(b@1 AS Float64) > random(), projection=[a@0] -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, b], file_type=parquet +01)FilterExec: CAST(b@1 AS Float64) > random(), projection=[a@0] +02)--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 ## cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/parquet_statistics.slt b/datafusion/sqllogictest/test_files/parquet_statistics.slt index c04235ef4ee6..14cf4b280247 100644 --- a/datafusion/sqllogictest/test_files/parquet_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_statistics.slt @@ -59,10 +59,9 @@ query TT EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] -02)--FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(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(121), [(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)], statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +01)FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +03)----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)], statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] # cleanup statement ok @@ -85,10 +84,9 @@ query TT EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] -02)--FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(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(121), [(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)], statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +01)FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +03)----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)], statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] # cleanup statement ok @@ -112,10 +110,9 @@ 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]: Min=Inexact(Int64(1)) Max=Inexact(Int64(1)))]] -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)], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +01)FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +03)----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)], 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 c10e67a22535..769a42108b14 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -672,16 +672,14 @@ physical_plan 02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------FilterExec: l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------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 +05)--------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 +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------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 +08)----CoalesceBatchesExec: target_batch_size=8192 +09)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +10)--------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 +11)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)------------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) @@ -771,10 +769,9 @@ physical_plan 10)----------------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 11)------------CoalesceBatchesExec: target_batch_size=8192 12)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -13)----------------CoalesceBatchesExec: target_batch_size=8192 -14)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -15)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)----------------------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 +13)----------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)--------------------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 # Simplification of a binary operator with a NULL value @@ -804,9 +801,8 @@ logical_plan 01)Filter: t.x < Int32(5) AND Boolean(NULL) 02)--TableScan: t projection=[x] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: x@0 < 5 AND NULL -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: x@0 < 5 AND NULL +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN FORMAT INDENT SELECT * FROM t WHERE x < 5 OR (10 * NULL < x); @@ -815,9 +811,8 @@ logical_plan 01)Filter: t.x < Int32(5) OR Boolean(NULL) 02)--TableScan: t projection=[x] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: x@0 < 5 OR NULL -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: x@0 < 5 OR NULL +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; @@ -846,9 +841,8 @@ logical_plan 01)Filter: t.x = Int32(5) 02)--TableScan: t projection=[x] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: x@0 = 5 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: x@0 = 5 +02)--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); diff --git a/datafusion/sqllogictest/test_files/projection.slt b/datafusion/sqllogictest/test_files/projection.slt index 9f840e7bdc2f..5a4411233424 100644 --- a/datafusion/sqllogictest/test_files/projection.slt +++ b/datafusion/sqllogictest/test_files/projection.slt @@ -276,7 +276,6 @@ logical_plan 03)----TableScan: t1 projection=[a], partial_filters=[t1.a > Int64(1)] physical_plan 01)ProjectionExec: expr=[] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@0 > 1 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection/17513.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 > 1, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 1, required_guarantees=[] +02)--FilterExec: a@0 > 1 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection/17513.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 > 1, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 1, required_guarantees=[] diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index 47095d92d937..cedc3f9f76a8 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -40,9 +40,8 @@ physical_plan 02)--UnnestExec 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------ProjectionExec: expr=[column2@0 as __unnest_placeholder(v.column2)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------FilterExec: column1@0 = 2, projection=[column2@1] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: column1@0 = 2, projection=[column2@1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query I select uc2 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3; @@ -56,12 +55,11 @@ explain select uc2 from (select unnest(column2) as uc2, column1 from v) where uc ---- physical_plan 01)ProjectionExec: expr=[__unnest_placeholder(v.column2,depth=1)@0 as uc2] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------UnnestExec -06)----------ProjectionExec: expr=[column2@0 as __unnest_placeholder(v.column2)] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------UnnestExec +05)--------ProjectionExec: expr=[column2@0 as __unnest_placeholder(v.column2)] +06)----------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; @@ -75,14 +73,12 @@ explain select uc2, column1 from (select unnest(column2) as uc2, column1 from v ---- physical_plan 01)ProjectionExec: expr=[__unnest_placeholder(v.column2,depth=1)@0 as uc2, column1@1 as column1] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 -04)------UnnestExec -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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] +02)--FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 +03)----UnnestExec +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] +06)----------FilterExec: column1@0 = 2 +07)------------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; @@ -97,12 +93,11 @@ explain select uc2, column1 from (select unnest(column2) as uc2, column1 from v ---- physical_plan 01)ProjectionExec: expr=[__unnest_placeholder(v.column2,depth=1)@0 as uc2, column1@1 as column1] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 OR column1@1 = 2 -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] +02)--FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 OR column1@1 = 2 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------UnnestExec +05)--------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table v; @@ -121,12 +116,11 @@ explain select * from (select column1, unnest(column2) as o from d) where o['a'] ---- physical_plan 01)ProjectionExec: expr=[column1@0 as column1, __unnest_placeholder(d.column2,depth=1)@1 as o] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: get_field(__unnest_placeholder(d.column2,depth=1)@1, a) = 1 -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] +02)--FilterExec: get_field(__unnest_placeholder(d.column2,depth=1)@1, a) = 1 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------UnnestExec +05)--------ProjectionExec: expr=[column1@0 as column1, column2@1 as __unnest_placeholder(d.column2)] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table d; @@ -143,12 +137,11 @@ query TT explain select * from (select unnest(column1) from d) where "__unnest_placeholder(d.column1).b" > 5; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: __unnest_placeholder(d.column1).b@1 > 5 -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------UnnestExec -05)--------ProjectionExec: expr=[column1@0 as __unnest_placeholder(d.column1)] -06)----------DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: __unnest_placeholder(d.column1).b@1 > 5 +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----UnnestExec +04)------ProjectionExec: expr=[column1@0 as __unnest_placeholder(d.column1)] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table d; diff --git a/datafusion/sqllogictest/test_files/pwmj.slt b/datafusion/sqllogictest/test_files/pwmj.slt index eafa4d0ba394..295eb94318ee 100644 --- a/datafusion/sqllogictest/test_files/pwmj.slt +++ b/datafusion/sqllogictest/test_files/pwmj.slt @@ -87,13 +87,11 @@ physical_plan 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----PiecewiseMergeJoin: operator=Gt, join_type=Inner, on=(t1_id > t2_id) 04)------SortExec: expr=[t1_id@0 ASC], preserve_partitioning=[false] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------FilterExec: t1_id@0 > 10 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------CoalesceBatchesExec: target_batch_size=8192 -10)----------FilterExec: t2_int@1 > 1, projection=[t2_id@0] -11)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: t1_id@0 > 10 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] +09)----------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.t1_id, t2.t2_id @@ -134,13 +132,11 @@ physical_plan 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----PiecewiseMergeJoin: operator=GtEq, join_type=Inner, on=(t1_id >= t2_id) 04)------SortExec: expr=[t1_id@0 ASC], preserve_partitioning=[false] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------FilterExec: t1_id@0 >= 22 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------CoalesceBatchesExec: target_batch_size=8192 -10)----------FilterExec: t2_int@1 = 3, projection=[t2_id@0] -11)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: t1_id@0 >= 22 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------FilterExec: t2_int@1 = 3, projection=[t2_id@0] +09)----------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.t1_id, t2.t2_id @@ -183,9 +179,8 @@ physical_plan 04)------SortExec: expr=[t1_id@0 DESC], preserve_partitioning=[false] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)--------CoalesceBatchesExec: target_batch_size=8192 -08)----------FilterExec: t2_int@1 >= 3, projection=[t2_id@0] -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------FilterExec: t2_int@1 >= 3, projection=[t2_id@0] +08)----------DataSourceExec: partitions=1, partition_sizes=[1] query II @@ -231,9 +226,8 @@ physical_plan 04)------SortExec: expr=[CAST(t1_id@0 AS Int64) DESC], preserve_partitioning=[false] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)--------CoalesceBatchesExec: target_batch_size=8192 -08)----------FilterExec: t2_int@1 >= 3, projection=[t2_id@0] -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------FilterExec: t2_int@1 >= 3, projection=[t2_id@0] +08)----------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.t1_id, t2.t2_id @@ -275,13 +269,11 @@ physical_plan 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----PiecewiseMergeJoin: operator=LtEq, join_type=Inner, on=(t1_id <= t2_id) 04)------SortExec: expr=[t1_id@0 DESC], preserve_partitioning=[false] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------FilterExec: t1_id@0 = 11 OR t1_id@0 = 44 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------CoalesceBatchesExec: target_batch_size=8192 -10)----------FilterExec: t2_name@1 != y, projection=[t2_id@0] -11)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: t1_id@0 = 11 OR t1_id@0 = 44 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------FilterExec: t2_name@1 != y, projection=[t2_id@0] +09)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok CREATE TABLE null_join_t1 (id INT); diff --git a/datafusion/sqllogictest/test_files/qualify.slt b/datafusion/sqllogictest/test_files/qualify.slt index 524f6baad2be..d4de5f9a9a61 100644 --- a/datafusion/sqllogictest/test_files/qualify.slt +++ b/datafusion/sqllogictest/test_files/qualify.slt @@ -273,10 +273,9 @@ logical_plan 05)--------TableScan: users projection=[id, name] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 > 1, projection=[id@0, name@1] -04)------WindowAggExec: wdw=[count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64 }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 > 1, projection=[id@0, name@1] +03)----WindowAggExec: wdw=[count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64 }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +04)------DataSourceExec: partitions=1, partition_sizes=[1] # plan row_number() query TT @@ -290,12 +289,11 @@ logical_plan 05)--------TableScan: users projection=[dept] physical_plan 01)ProjectionExec: expr=[row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@0 as rk] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@0 > 1 -04)------ProjectionExec: expr=[row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] -05)--------BoundedWindowAggExec: wdw=[row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -06)----------SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[false] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@0 > 1 +03)----ProjectionExec: expr=[row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +04)------BoundedWindowAggExec: wdw=[row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +05)--------SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[false] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] # plan with window function and group by query TT @@ -318,23 +316,20 @@ logical_plan 09)----------------TableScan: users projection=[salary, dept] physical_plan 01)ProjectionExec: expr=[dept@0 as dept, avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as r] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 > Some(60000000000),14,6 -04)------ProjectionExec: expr=[dept@0 as dept, avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] -05)--------WindowAggExec: wdw=[avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(14, 6), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -06)----------SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[true] -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=4 -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------FilterExec: sum(users.salary)@2 > Some(2000000),20,2, projection=[dept@0, salary@1] -11)--------------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept, salary@1 as salary], aggr=[sum(users.salary)] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([dept@0, salary@1], 4), input_partitions=4 -14)--------------------------AggregateExec: mode=Partial, gby=[dept@1 as dept, salary@0 as salary], aggr=[sum(users.salary)] -15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------FilterExec: salary@0 > Some(500000),10,2 -18)----------------------------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 > Some(60000000000),14,6 +03)----ProjectionExec: expr=[dept@0 as dept, avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +04)------WindowAggExec: wdw=[avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(14, 6), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +05)--------SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[true] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=4 +08)--------------FilterExec: sum(users.salary)@2 > Some(2000000),20,2, projection=[dept@0, salary@1] +09)----------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept, salary@1 as salary], aggr=[sum(users.salary)] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([dept@0, salary@1], 4), input_partitions=4 +12)----------------------AggregateExec: mode=Partial, gby=[dept@1 as dept, salary@0 as salary], aggr=[sum(users.salary)] +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------------------FilterExec: salary@0 > Some(500000),10,2 +15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] # plan with aggregate function query TT @@ -355,17 +350,16 @@ physical_plan 01)SortPreservingMergeExec: [dept@0 ASC NULLS LAST] 02)--SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[dept@0 as dept, sum(users.salary)@1 as s] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------FilterExec: rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 = 1, projection=[dept@0, sum(users.salary)@1] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------BoundedWindowAggExec: wdw=[rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -08)--------------SortPreservingMergeExec: [sum(users.salary)@1 DESC] -09)----------------SortExec: expr=[sum(users.salary)@1 DESC], preserve_partitioning=[true] -10)------------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept], aggr=[sum(users.salary)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=1 -13)------------------------AggregateExec: mode=Partial, gby=[dept@1 as dept], aggr=[sum(users.salary)] -14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------FilterExec: rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 = 1, projection=[dept@0, sum(users.salary)@1] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------BoundedWindowAggExec: wdw=[rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +07)------------SortPreservingMergeExec: [sum(users.salary)@1 DESC] +08)--------------SortExec: expr=[sum(users.salary)@1 DESC], preserve_partitioning=[true] +09)----------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept], aggr=[sum(users.salary)] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=1 +12)----------------------AggregateExec: mode=Partial, gby=[dept@1 as dept], aggr=[sum(users.salary)] +13)------------------------DataSourceExec: partitions=1, partition_sizes=[1] # Clean up statement ok diff --git a/datafusion/sqllogictest/test_files/regexp/regexp_like.slt b/datafusion/sqllogictest/test_files/regexp/regexp_like.slt index dd42511eade9..6f2d5a873c1b 100644 --- a/datafusion/sqllogictest/test_files/regexp/regexp_like.slt +++ b/datafusion/sqllogictest/test_files/regexp/regexp_like.slt @@ -251,9 +251,8 @@ logical_plan 01)Filter: dict_table.column1 LIKE Utf8("%oo%") 02)--TableScan: dict_table projection=[column1] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column1@0 LIKE %oo% -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: column1@0 LIKE %oo% +02)--DataSourceExec: partitions=1, partition_sizes=[1] # Ensure casting / coercion works for all operators # (there should be no casts to Utf8) diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index a3b6b380c57f..f6061b61a249 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -121,10 +121,9 @@ logical_plan 03)----TableScan: sink_table projection=[c1, c2, c3] physical_plan 01)CoalescePartitionsExec: fetch=5 -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 +02)--FilterExec: c3@2 > 0, fetch=5 +03)----RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 +04)------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 41718b3aebc2..06ea22761d92 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -59,9 +59,8 @@ logical_plan 01)Filter: parquet_table.column1 != Int32(42) 02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] 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..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, 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)] +01)FilterExec: column1@0 != 42 +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, 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 @@ -75,9 +74,8 @@ logical_plan 01)Filter: parquet_table.column1 != Int32(42) 02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] 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..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, 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)] +01)FilterExec: column1@0 != 42 +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, 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 @@ -100,9 +98,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 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..266], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:266..526, 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..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:272..537]]}, 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)----FilterExec: column1@0 != 42 +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..266], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:266..526, 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..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:272..537]]}, 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 @@ -136,9 +133,8 @@ logical_plan 03)----TableScan: parquet_table_with_order projection=[column1], partial_filters=[parquet_table_with_order.column1 != Int32(42)] 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..263], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..268], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:268..537], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:263..526]]}, 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)] +02)--FilterExec: column1@0 != 42 +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..263], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..268], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:268..537], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:263..526]]}, 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 @@ -183,9 +179,8 @@ logical_plan 01)Filter: csv_table.column1 != Int32(42) 02)--TableScan: csv_table projection=[column1], partial_filters=[csv_table.column1 != Int32(42)] 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 +01)FilterExec: column1@0 != 42 +02)--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 @@ -226,9 +221,8 @@ logical_plan 01)Filter: json_table.column1 != Int32(42) 02)--TableScan: json_table projection=[column1], partial_filters=[json_table.column1 != Int32(42)] 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 +01)FilterExec: column1@0 != 42 +02)--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 diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 5c684eb83d1a..21878d36db02 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1440,10 +1440,9 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------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 +02)--FilterExec: a@1 = 0 +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 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 @@ -1461,10 +1460,9 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------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 +02)--FilterExec: a@1 = 0 AND b@2 = 0 +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 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 @@ -1482,10 +1480,9 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------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 +02)--FilterExec: a@1 = 0 AND b@2 = 0 +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 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 @@ -1503,10 +1500,9 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan 01)SortPreservingMergeExec: [a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------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 +02)--FilterExec: a@1 = 0 AND b@2 = 0 +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 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 @@ -1525,10 +1521,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] 02)--SortExec: expr=[c@3 ASC NULLS LAST], preserve_partitioning=[true] -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 +03)----FilterExec: a@1 = 0 OR 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 # When ordering lost during projection, we shouldn't keep the SortExec. # in the final physical plan. @@ -1573,10 +1568,9 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) = annotated_data_finite2.a] physical_plan 01)SortPreservingMergeExec: [CAST(round(CAST(b@2 AS Float64)) AS Int32) ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------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 +02)--FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 +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 NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true statement ok diff --git a/datafusion/sqllogictest/test_files/simplify_expr.slt b/datafusion/sqllogictest/test_files/simplify_expr.slt index 2387385369cb..d8c25ab25e8e 100644 --- a/datafusion/sqllogictest/test_files/simplify_expr.slt +++ b/datafusion/sqllogictest/test_files/simplify_expr.slt @@ -26,9 +26,8 @@ logical_plan 01)Filter: t.a = Int32(3) 02)--TableScan: t projection=[a] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: a@0 = 3 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: a@0 = 3 +02)--DataSourceExec: partitions=1, partition_sizes=[1] # test regex exprs query TT @@ -38,9 +37,8 @@ logical_plan 01)Filter: t.b ~ Utf8View(".*") 02)--TableScan: t projection=[b] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: b@0 ~ .* -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: b@0 ~ .* +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select b from t where b !~ '.*' @@ -49,9 +47,8 @@ logical_plan 01)Filter: t.b !~ Utf8View(".*") 02)--TableScan: t projection=[b] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: b@0 !~ .* -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: b@0 !~ .* +02)--DataSourceExec: partitions=1, partition_sizes=[1] query T select b from t where b ~ '.*' @@ -70,9 +67,8 @@ logical_plan 01)Filter: t.a IS NOT NULL OR Boolean(NULL) 02)--TableScan: t projection=[a, b] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: a@0 IS NOT NULL OR NULL -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: a@0 IS NOT NULL OR NULL +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; @@ -117,4 +113,3 @@ logical_plan physical_plan 01)ProjectionExec: expr=[[{x:100}] as a] 02)--PlaceholderRowExec - diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index dec935749535..27325d4e5e84 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -311,16 +311,15 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -06)----------CoalesceBatchesExec: target_batch_size=2 -07)------------FilterExec: sum(t2.t2_int)@1 < 3 -08)--------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -09)----------------CoalesceBatchesExec: target_batch_size=2 -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=[2] -14)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)--------DataSourceExec: partitions=1, partition_sizes=[2] +06)----------FilterExec: sum(t2.t2_int)@1 < 3 +07)------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +08)--------------CoalesceBatchesExec: target_batch_size=2 +09)----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +10)------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------DataSourceExec: partitions=1, partition_sizes=[2] +13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------DataSourceExec: partitions=1, partition_sizes=[2] 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 @@ -1189,13 +1188,12 @@ logical_plan 05)------SubqueryAlias: __correlated_sq_1 06)--------TableScan: t2 projection=[t2_id] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -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=RightMark, on=[(t2_id@0, t1_id@0)] -05)--------DataSourceExec: partitions=1, partition_sizes=[2] -06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[2] +01)FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(t2_id@0, t1_id@0)] +04)------DataSourceExec: partitions=1, partition_sizes=[2] +05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[2] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index 4a6ad5eddfb7..6d76be67c206 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -55,6 +55,5 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] 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 +09)----------------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] +10)------------------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 4cfd69bbc24f..f848a330b276 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -91,14 +91,12 @@ physical_plan 20)--------------------------------------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 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 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -25)------------------------------------------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 -26)--------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -28)------------------------------CoalesceBatchesExec: target_batch_size=8192 -29)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -30)----------------------------------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 -31)------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -33)----------------------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 +23)--------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +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], file_type=csv, has_header=false +25)--------------------------CoalesceBatchesExec: target_batch_size=8192 +26)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +27)------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +28)--------------------------------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 +29)------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +31)----------------------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 52bbd7b63afb..6d3831cfaeaf 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -96,29 +96,27 @@ physical_plan 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([s_suppkey@0], 4), input_partitions=1 -43)------------------------------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 -44)------------------CoalesceBatchesExec: target_batch_size=8192 -45)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -46)----------------------CoalesceBatchesExec: target_batch_size=8192 -47)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -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 +24)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +25)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +26)------------------------------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 +27)------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)] +28)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +29)----------CoalescePartitionsExec +30)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +31)--------------CoalesceBatchesExec: target_batch_size=8192 +32)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +33)------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +35)----------------------CoalesceBatchesExec: target_batch_size=8192 +36)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +37)--------------------------CoalesceBatchesExec: target_batch_size=8192 +38)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +39)------------------------------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 +40)--------------------------CoalesceBatchesExec: target_batch_size=8192 +41)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +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([n_nationkey@0], 4), input_partitions=4 +45)----------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +46)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index f7344daed8c7..c2bc4d8511ef 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -70,9 +70,8 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -12)----------------------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 +12)----------------------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] +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_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false +14)------------------CoalesceBatchesExec: target_batch_size=8192 +15)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +16)----------------------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 17d827cebb82..65531be833fd 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -69,6 +69,5 @@ physical_plan 14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false 15)----------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -17)--------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -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_custkey, o_comment], file_type=csv, has_header=false +17)--------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +18)----------------------------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 71dea1a5e12a..32e05fa583d6 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -50,9 +50,8 @@ physical_plan 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] 08)--------------CoalesceBatchesExec: target_batch_size=8192 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=1 -15)------------------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 +10)------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] +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_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +12)--------------CoalesceBatchesExec: target_batch_size=8192 +13)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=1 +14)------------------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 a3284b484122..d7275347bd1e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -83,19 +83,17 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 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=1 -20)----------------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 -21)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -22)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -23)----------------CoalesceBatchesExec: target_batch_size=8192 -24)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -25)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -26)----------------------CoalesceBatchesExec: target_batch_size=8192 -27)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -28)--------------------------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 +13)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +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_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +15)--------CoalesceBatchesExec: target_batch_size=8192 +16)----------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] +17)------------CoalesceBatchesExec: target_batch_size=8192 +18)--------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +19)----------------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 +20)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +21)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +22)----------------CoalesceBatchesExec: target_batch_size=8192 +23)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +24)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +25)----------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +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_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 53d637ea3f51..fca427dd8632 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -87,13 +87,11 @@ physical_plan 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 p_size@3 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 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 +23)--------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND p_size@3 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 9]) +24)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +25)------------------------------------------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 +26)--------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +28)------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] +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_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..e2123cfc7e56 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -64,13 +64,12 @@ physical_plan 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 +14)--------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] +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_brand, p_container], file_type=csv, has_header=false +17)------------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] +18)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +19)----------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +21)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +22)----------------------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 16a5b7eb39b5..9225f21a61e5 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -90,10 +90,9 @@ physical_plan 21)--------------------CoalesceBatchesExec: target_batch_size=8192 22)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 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_orderkey, l_quantity], file_type=csv, has_header=false -24)----------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] -26)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -27)----------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -29)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -30)----------------------------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 +24)----------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] +25)------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +26)--------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +28)------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +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_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 4960ad1f4a91..12efc64555b2 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -72,12 +72,10 @@ physical_plan 06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN ([SM CASE, SM BOX, SM PACK, SM PKG]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([MED BAG, MED BOX, MED PKG, MED PACK]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([LG CASE, LG BOX, LG PACK, LG PKG]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -09)----------------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 ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([LG CASE, LG BOX, LG PACK, 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 +09)----------------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] +10)------------------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 +11)------------CoalesceBatchesExec: target_batch_size=8192 +12)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +13)----------------FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@2 <= 15) AND p_size@2 >= 1 +14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)--------------------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 c299fa88a9c4..bf412998b6f4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -123,54 +123,51 @@ physical_plan 22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] 23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)----------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -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=1 -34)----------------------------------------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 -35)--------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -37)------------------------------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 -38)------------------CoalesceBatchesExec: target_batch_size=8192 -39)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -40)----------------------CoalesceBatchesExec: target_batch_size=8192 -41)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -42)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -43)----------------------------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 -44)----------CoalesceBatchesExec: target_batch_size=8192 -45)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -46)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -47)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -48)------------------CoalesceBatchesExec: target_batch_size=8192 -49)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -50)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -51)------------------------CoalesceBatchesExec: target_batch_size=8192 -52)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -53)----------------------------CoalesceBatchesExec: target_batch_size=8192 -54)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -55)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -56)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -57)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -59)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -60)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -61)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -62)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -63)------------------------------------------------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 -64)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -65)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -66)------------------------------------------------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 -67)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -68)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -69)----------------------------------------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 -70)----------------------------CoalesceBatchesExec: target_batch_size=8192 -71)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -72)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -73)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -74)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -75)--------------------------------------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 +25)------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] +26)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)----------------------------------------------------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 +28)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +29)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +30)------------------------------------------------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 +31)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +33)----------------------------------------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 +34)--------------------------CoalesceBatchesExec: target_batch_size=8192 +35)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +36)------------------------------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 +37)------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +39)----------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +40)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +41)--------------------------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 +42)----------CoalesceBatchesExec: target_batch_size=8192 +43)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +44)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +45)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +46)------------------CoalesceBatchesExec: target_batch_size=8192 +47)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +48)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +49)------------------------CoalesceBatchesExec: target_batch_size=8192 +50)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +51)----------------------------CoalesceBatchesExec: target_batch_size=8192 +52)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +53)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +54)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +55)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +56)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +57)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +59)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +60)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +61)------------------------------------------------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 +62)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +63)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +64)------------------------------------------------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 +65)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +66)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +67)----------------------------------------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 +68)----------------------------CoalesceBatchesExec: target_batch_size=8192 +69)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +70)--------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +71)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +72)------------------------------------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 492c68d6aaa0..9dc0f6822a1a 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -94,32 +94,29 @@ physical_plan 11)--------------------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 12)----------------CoalesceBatchesExec: target_batch_size=8192 13)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -14)--------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] -16)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------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 -18)--------CoalesceBatchesExec: target_batch_size=8192 -19)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -20)------------CoalesceBatchesExec: target_batch_size=8192 -21)--------------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] -22)----------------CoalesceBatchesExec: target_batch_size=8192 -23)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -24)--------------------CoalesceBatchesExec: target_batch_size=8192 -25)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -26)------------------------CoalesceBatchesExec: target_batch_size=8192 -27)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -28)----------------------------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 -29)------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -31)----------------------------CoalesceBatchesExec: target_batch_size=8192 -32)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] -33)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -34)----------------------------------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 -35)----------------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] -36)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -37)--------------------CoalesceBatchesExec: target_batch_size=8192 -38)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -39)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -40)--------------------------CoalesceBatchesExec: target_batch_size=8192 -41)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] -42)------------------------------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 +14)--------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] +15)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------------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 +17)--------CoalesceBatchesExec: target_batch_size=8192 +18)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +19)------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------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] +21)----------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +23)--------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +25)------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +27)----------------------------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 +28)------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +30)----------------------------FilterExec: p_name@1 LIKE forest%, 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)----------------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] +34)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +35)--------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +37)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +38)--------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] +39)----------------------------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 96341ba32311..adb1b9bd10cc 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -118,25 +118,21 @@ physical_plan 26)--------------------------------------------------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 27)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -29)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -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_suppkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false -32)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -34)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] -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_orderstatus], file_type=csv, has_header=false -37)------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -39)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] -41)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -42)----------------------------------------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 -43)----------------------CoalesceBatchesExec: target_batch_size=8192 -44)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -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_suppkey], file_type=csv, has_header=false -46)------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -48)----------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -50)--------------------------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 +29)--------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +30)----------------------------------------------------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 +31)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +33)------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] +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_orderstatus], file_type=csv, has_header=false +35)------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +37)----------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] +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], file_type=csv, has_header=false +40)----------------------CoalesceBatchesExec: target_batch_size=8192 +41)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +42)--------------------------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 +43)------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +45)----------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +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, 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 22476156b80d..818c7bc98965 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -90,17 +90,15 @@ physical_plan 14)--------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------FilterExec: substr(c_phone@1, 1, 2) IN ([13, 31, 23, 29, 30, 18, 17]) -19)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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)--------------------------------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 -24)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] -25)----------------------CoalescePartitionsExec -26)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -27)--------------------------CoalesceBatchesExec: target_batch_size=8192 -28)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] -29)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)--------------------------------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 +17)--------------------------------FilterExec: substr(c_phone@1, 1, 2) IN ([13, 31, 23, 29, 30, 18, 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)--------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] +27)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)------------------------------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..27fd404644fa 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -73,17 +73,14 @@ physical_plan 13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -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 +16)------------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] +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_mktsegment], file_type=csv, has_header=false +19)--------------------------CoalesceBatchesExec: target_batch_size=8192 +20)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +21)------------------------------FilterExec: o_orderdate@2 < 1995-03-15 +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_orderdate, o_shippriority], file_type=csv, has_header=false +23)------------------CoalesceBatchesExec: target_batch_size=8192 +24)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +25)----------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +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_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..9f53448304a6 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -64,11 +64,9 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------------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 +12)----------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] +13)------------------------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 +14)------------------CoalesceBatchesExec: target_batch_size=8192 +15)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +16)----------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] +17)------------------------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 dcf462915899..5c2fa1a40898 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -97,21 +97,19 @@ physical_plan 28)------------------------------------------------------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 29)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 30)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -31)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -33)----------------------------------------------------------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 -34)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -36)----------------------------------------------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 -37)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 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([n_nationkey@0], 4), input_partitions=1 -42)------------------------------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 -43)------------------CoalesceBatchesExec: target_batch_size=8192 -44)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -45)----------------------CoalesceBatchesExec: target_batch_size=8192 -46)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] -47)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -48)----------------------------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 +31)------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +32)--------------------------------------------------------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 +33)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 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_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false +36)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 +38)--------------------------------------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 +39)--------------------------CoalesceBatchesExec: target_batch_size=8192 +40)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +41)------------------------------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 +42)------------------CoalesceBatchesExec: target_batch_size=8192 +43)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +44)----------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +45)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +46)--------------------------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..eb9063d69171 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part @@ -38,6 +38,5 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] -05)--------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 +05)--------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] +06)----------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 53ab43ba491b..e19ae5c350e9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -115,24 +115,21 @@ physical_plan 29)--------------------------------------------------------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 30)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 31)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -32)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)----------------------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 -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_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false -35)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -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_custkey], file_type=csv, has_header=false -38)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -40)----------------------------------------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 -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 = FRANCE OR n_name@1 = GERMANY -45)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -46)--------------------------------------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 -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 = GERMANY OR n_name@1 = FRANCE -51)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -52)------------------------------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)--------------------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 +33)----------------------------------------------------------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 +34)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +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], file_type=csv, has_header=false +37)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +39)----------------------------------------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 +40)----------------------------CoalesceBatchesExec: target_batch_size=8192 +41)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +42)--------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +43)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +44)------------------------------------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 +45)--------------------CoalesceBatchesExec: target_batch_size=8192 +46)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +47)------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +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 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 6b0b05f2f636..5d3ce5e56611 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -126,33 +126,30 @@ physical_plan 34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] 35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -37)------------------------------------------------------------------------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=1 -46)----------------------------------------------------------------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 -47)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -48)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -49)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -50)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -51)------------------------------------------------------------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 -52)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -53)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -54)------------------------------------------------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 -55)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -56)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -57)----------------------------------------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 -58)----------------------------CoalesceBatchesExec: target_batch_size=8192 -59)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -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 -61)--------------------CoalesceBatchesExec: target_batch_size=8192 -62)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -63)------------------------CoalesceBatchesExec: target_batch_size=8192 -64)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] -65)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -66)------------------------------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 +37)------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] +38)--------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +39)----------------------------------------------------------------------------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 +40)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +42)------------------------------------------------------------------------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 +43)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +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([o_orderkey@0], 4), input_partitions=4 +48)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +49)----------------------------------------------------------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 +50)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +51)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +52)------------------------------------------------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 +53)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +54)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +55)----------------------------------------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 +56)----------------------------CoalesceBatchesExec: target_batch_size=8192 +57)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +58)--------------------------------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 +59)--------------------CoalesceBatchesExec: target_batch_size=8192 +60)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +61)------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] +62)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +63)----------------------------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 dcee31dfecd3..e4aa46dc15d1 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -103,22 +103,21 @@ physical_plan 26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] 27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -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=1 -38)------------------------------------------------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 -39)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -41)----------------------------------------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 -42)----------------------------CoalesceBatchesExec: target_batch_size=8192 -43)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -44)--------------------------------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 -45)--------------------CoalesceBatchesExec: target_batch_size=8192 -46)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -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 +29)--------------------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] +30)----------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +31)------------------------------------------------------------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 +32)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=csv, has_header=false +35)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +37)------------------------------------------------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 +38)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@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_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false +41)----------------------------CoalesceBatchesExec: target_batch_size=8192 +42)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +43)--------------------------------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 +44)--------------------CoalesceBatchesExec: target_batch_size=8192 +45)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +46)------------------------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/union.slt b/datafusion/sqllogictest/test_files/union.slt index f7ab6a0c9281..9e63f79f4545 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -513,16 +513,15 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=2 10)------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -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: Field { "lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": nullable Int64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -20)--------ProjectionExec: expr=[1 as c1] -21)----------PlaceholderRowExec +12)----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true +15)----ProjectionExec: expr=[1 as cnt] +16)------PlaceholderRowExec +17)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] +18)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": nullable Int64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +19)--------ProjectionExec: expr=[1 as c1] +20)----------PlaceholderRowExec ######## @@ -834,14 +833,12 @@ logical_plan physical_plan 01)CoalescePartitionsExec 02)--UnionExec -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 +03)----FilterExec: c1@0 = a +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------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)----FilterExec: c1@0 = a +07)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------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 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 26cb71acbdfe..2e228472d68c 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1885,10 +1885,9 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=4096 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -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 +10)------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +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, c13], file_type=csv, has_header=true query I @@ -3219,10 +3218,9 @@ logical_plan 06)----------TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan 01)ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as rn1] -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: Field { "row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], 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] +02)--FilterExec: row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 < 50, fetch=5 +03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +04)------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 @@ -3525,9 +3523,8 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] 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: Field { "sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Int64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], 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 +02)--FilterExec: b@2 = 0 +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 # Since column b is constant after filter b=0, # window requirement b ASC, d ASC can be satisfied @@ -3544,9 +3541,8 @@ logical_plan physical_plan 01)BoundedWindowAggExec: wdw=[sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Int64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 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 +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 # Create an unbounded source where there is multiple orderings. @@ -3599,9 +3595,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] 02)--BoundedWindowAggExec: wdw=[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "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": nullable Int32 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], 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 +03)----FilterExec: d@1 = 0 +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], file_type=csv, has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c ORDER BY a ASC) @@ -4216,9 +4211,8 @@ physical_plan 02)--BoundedWindowAggExec: wdw=[count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": Int64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 -05)--------CoalesceBatchesExec: target_batch_size=4096 -06)----------FilterExec: a@0 = 1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: a@0 = 1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4238,9 +4232,8 @@ physical_plan 02)--BoundedWindowAggExec: wdw=[row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 -05)--------CoalesceBatchesExec: target_batch_size=4096 -06)----------FilterExec: a@0 = 1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: a@0 = 1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT @@ -5307,10 +5300,9 @@ physical_plan 04)------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=1 06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -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] +07)------------FilterExec: c1@0 = 2 OR c1@0 = 3 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5345,13 +5337,12 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, rank@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rank] -03)----CoalesceBatchesExec: target_batch_size=1 -04)------FilterExec: c2@1 >= 10 -05)--------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] -07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +03)----FilterExec: c2@1 >= 10 +04)------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] +06)----------CoalesceBatchesExec: target_batch_size=1 +07)------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5386,16 +5377,14 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, rank@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rank] -03)----CoalesceBatchesExec: target_batch_size=1 -04)------FilterExec: c2@1 = 10 -05)--------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -06)----------SortExec: expr=[c2@1 ASC NULLS LAST], preserve_partitioning=[true] -07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -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] +03)----FilterExec: c2@1 = 10 +04)------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +05)--------SortExec: expr=[c2@1 ASC NULLS LAST], preserve_partitioning=[true] +06)----------CoalesceBatchesExec: target_batch_size=1 +07)------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +08)--------------FilterExec: c1@0 = 1 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5427,13 +5416,12 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, rank@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rank] -03)----CoalesceBatchesExec: target_batch_size=1 -04)------FilterExec: c1@0 = 1 OR c2@1 = 10 -05)--------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] -07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +03)----FilterExec: c1@0 = 1 OR c2@1 = 10 +04)------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] +06)----------CoalesceBatchesExec: target_batch_size=1 +07)------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5479,10 +5467,9 @@ physical_plan 09)----------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 10)------------------CoalesceBatchesExec: target_batch_size=1 11)--------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -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] +12)----------------------FilterExec: c1@0 > 1 +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2 @@ -5526,13 +5513,12 @@ physical_plan 05)--------SortExec: expr=[c2@1 ASC NULLS LAST, c1@0 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=1 07)------------RepartitionExec: partitioning=Hash([c2@1, c1@0], 2), input_partitions=2 -08)--------------CoalesceBatchesExec: target_batch_size=1 -09)----------------FilterExec: c2@1 > 1 -10)------------------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -11)--------------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] -12)----------------------CoalesceBatchesExec: target_batch_size=1 -13)------------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 -14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------FilterExec: c2@1 > 1 +09)----------------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +10)------------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] +11)--------------------CoalesceBatchesExec: target_batch_size=1 +12)----------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +13)------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2 From 0af2832d857f4df735ab4ba4d801fa13c914c0fe Mon Sep 17 00:00:00 2001 From: Kumar Ujjawal Date: Fri, 14 Nov 2025 20:18:43 +0530 Subject: [PATCH 09/25] minor: refactor with `assert_or_internal_err!()` in `datafusion/datasource` (#18697) ## Which issue does this PR close? - Part of https://github.com/apache/datafusion/issues/18613 ## Rationale for this change ## What changes are included in this PR? ## Are these changes tested? ## Are there any user-facing changes? --- datafusion/datasource/src/memory.rs | 16 +++++++++------- datafusion/datasource/src/sink.rs | 10 ++++++---- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 7d5c8c4834ea..9bdbf433bd2f 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -29,7 +29,10 @@ use crate::source::{DataSource, DataSourceExec}; use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::{Schema, SchemaRef}; -use datafusion_common::{internal_err, plan_err, project_schema, Result, ScalarValue}; +use datafusion_common::{ + assert_or_internal_err, plan_err, project_schema, DataFusionError, Result, + ScalarValue, +}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::project_orderings; use datafusion_physical_expr::utils::collect_columns; @@ -438,12 +441,11 @@ impl MemorySourceConfig { .map(|field| field.name() != col.name()) .unwrap_or(true) }); - if let Some(col) = ambiguous_column { - return internal_err!( - "Column {:?} is not found in the original schema of the MemorySourceConfig", - col - ); - } + assert_or_internal_err!( + ambiguous_column.is_none(), + "Column {:?} is not found in the original schema of the MemorySourceConfig", + ambiguous_column.as_ref().unwrap() + ); // If there is a projection on the source, we also need to project orderings if self.projection.is_some() { diff --git a/datafusion/datasource/src/sink.rs b/datafusion/datasource/src/sink.rs index a8adb46b96ff..a4ab78d07840 100644 --- a/datafusion/datasource/src/sink.rs +++ b/datafusion/datasource/src/sink.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use arrow::array::{ArrayRef, RecordBatch, UInt64Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::{internal_err, Result}; +use datafusion_common::{assert_eq_or_internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{Distribution, EquivalenceProperties}; use datafusion_physical_expr_common::sort_expr::{LexRequirement, OrderingRequirements}; @@ -226,9 +226,11 @@ impl ExecutionPlan for DataSinkExec { partition: usize, context: Arc, ) -> Result { - if partition != 0 { - return internal_err!("DataSinkExec can only be called on partition 0!"); - } + assert_eq_or_internal_err!( + partition, + 0, + "DataSinkExec can only be called on partition 0!" + ); let data = execute_input_stream( Arc::clone(&self.input), Arc::clone(self.sink.schema()), From 244456faeb0e6fa97bb324960ba33b7f03620315 Mon Sep 17 00:00:00 2001 From: Aryan Bagade <73382554+AryanBagade@users.noreply.github.com> Date: Fri, 14 Nov 2025 07:15:07 -0800 Subject: [PATCH 10/25] chore: Enforce lint rule `clippy::needless_pass_by_value` to datafusion-datasource (#18682) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Rationale for this change This PR enforces the `clippy::needless_pass_by_value` lint rule to prevent unnecessary data clones and improve performance in the `datafusion-datasource` crate. This is part of the effort tracked in #18503 to enforce this lint rule across all DataFusion crates. Functions that take ownership of values (pass-by-value) when they only need to read them force callers to `.clone()` data unnecessarily, which degrades performance. By changing these functions to accept references instead, we eliminate these unnecessary clones. ## What changes are included in this PR? - Added lint rule enforcement to `datafusion/datasource/src/mod.rs` - Fixed 11 violations of `clippy::needless_pass_by_value` across 5 files: - `file_scan_config.rs`: 2 fixes - `memory.rs`: 3 fixes - `source.rs`: 1 fix - `statistics.rs`: 4 fixes - `write/demux.rs`: 1 fix - Updated callers in `datafusion-core` and `datafusion-catalog-listing` to pass references ## Are these changes tested? Yes, all changes are tested: - ✅ All 82 unit tests pass (`cargo test -p datafusion-datasource`) - ✅ All 7 doc tests pass - ✅ Strict clippy checks pass with `-D warnings` - ✅ CI lint script passes (`./dev/rust_lint.sh`) - ✅ Dependent crates (`datafusion-catalog-listing`, `datafusion-core`) pass all tests and clippy checks Tests are covered by existing tests as this is a refactoring that changes internal function signatures without changing behavior. ## Are there any user-facing changes? No user-facing changes. All changes are internal to the `datafusion-datasource` crate. The public API remains unchanged - only internal function signatures were modified to accept references instead of owned values. Then at the bottom add: Fixes #18611 Part of #18503 --- datafusion/datasource/src/file_scan_config.rs | 8 ++-- datafusion/datasource/src/memory.rs | 9 +++-- datafusion/datasource/src/mod.rs | 4 ++ datafusion/datasource/src/source.rs | 9 ++--- datafusion/datasource/src/statistics.rs | 39 +++++++++---------- datafusion/datasource/src/write/demux.rs | 4 +- 6 files changed, 39 insertions(+), 34 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 82052ee4c39c..897206c3d56e 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -581,7 +581,8 @@ impl DataSource for FileScanConfig { if let Some(filter) = self.file_source.filter() { // We need to remap column indexes to match the projected schema since that's what the equivalence properties deal with. // Note that this will *ignore* any non-projected columns: these don't factor into ordering / equivalence. - match Self::add_filter_equivalence_info(filter, &mut eq_properties, &schema) { + match Self::add_filter_equivalence_info(&filter, &mut eq_properties, &schema) + { Ok(()) => {} Err(e) => { warn!("Failed to add filter equivalence info: {e}"); @@ -782,12 +783,12 @@ impl FileScanConfig { } fn add_filter_equivalence_info( - filter: Arc, + filter: &Arc, eq_properties: &mut EquivalenceProperties, schema: &Schema, ) -> Result<()> { // Gather valid equality pairs from the filter expression - let equal_pairs = split_conjunction(&filter).into_iter().filter_map(|expr| { + let equal_pairs = split_conjunction(filter).into_iter().filter_map(|expr| { // Ignore any binary expressions that reference non-existent columns in the current schema // (e.g. due to unnecessary projections being removed) reassign_expr_columns(Arc::clone(expr), schema) @@ -1145,6 +1146,7 @@ impl PartitionColumnProjector { // to the right positions as deduced from `projected_schema` // - `file_batch`: batch read from the file, with internal projection applied // - `partition_values`: the list of partition values, one for each partition column + #[expect(clippy::needless_pass_by_value)] pub fn project( &mut self, file_batch: RecordBatch, diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 9bdbf433bd2f..036574ddd3c3 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -285,6 +285,7 @@ impl MemorySourceConfig { } /// Create a new execution plan from a list of constant values (`ValuesExec`) + #[expect(clippy::needless_pass_by_value)] pub fn try_new_as_values( schema: SchemaRef, data: Vec>>, @@ -342,6 +343,7 @@ impl MemorySourceConfig { /// /// Errors if any of the batches don't match the provided schema, or if no /// batches are provided. + #[expect(clippy::needless_pass_by_value)] pub fn try_new_from_batches( schema: SchemaRef, batches: Vec, @@ -945,10 +947,9 @@ mod tests { vec![lit(ScalarValue::Null)], ]; let rows = data.len(); - let values = MemorySourceConfig::try_new_as_values( - Arc::new(Schema::new(vec![Field::new("col0", DataType::Null, true)])), - data, - )?; + let schema = + Arc::new(Schema::new(vec![Field::new("col0", DataType::Null, true)])); + let values = MemorySourceConfig::try_new_as_values(schema, data)?; assert_eq!( values.partition_statistics(None)?, diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index 8d988bdb31be..2c7d40d2fb3b 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -23,6 +23,10 @@ // Make sure fast / cheap clones on Arc are explicit: // https://github.com/apache/datafusion/issues/11143 #![cfg_attr(not(test), deny(clippy::clone_on_ref_ptr))] +// Enforce lint rule to prevent needless pass by value +// https://github.com/apache/datafusion/issues/18503 +#![deny(clippy::needless_pass_by_value)] +#![cfg_attr(test, allow(clippy::needless_pass_by_value))] //! A table that uses the `ObjectStore` listing capability //! to get the list of files to process. diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 11a8a3867b80..de79512a4101 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -348,8 +348,7 @@ impl ExecutionPlan for DataSourceExec { let mut new_node = self.clone(); new_node.data_source = data_source; // Re-compute properties since we have new filters which will impact equivalence info - new_node.cache = - Self::compute_properties(Arc::clone(&new_node.data_source)); + new_node.cache = Self::compute_properties(&new_node.data_source); Ok(FilterPushdownPropagation { filters: res.filters, @@ -371,7 +370,7 @@ impl DataSourceExec { // 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)); + let cache = Self::compute_properties(&data_source); Self { data_source, cache } } @@ -381,7 +380,7 @@ impl DataSourceExec { } pub fn with_data_source(mut self, data_source: Arc) -> Self { - self.cache = Self::compute_properties(Arc::clone(&data_source)); + self.cache = Self::compute_properties(&data_source); self.data_source = data_source; self } @@ -398,7 +397,7 @@ impl DataSourceExec { self } - fn compute_properties(data_source: Arc) -> PlanProperties { + fn compute_properties(data_source: &Arc) -> PlanProperties { PlanProperties::new( data_source.eq_properties(), data_source.output_partitioning(), diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index 0dd9bdb87c40..980677e488b8 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -152,28 +152,25 @@ impl MinMaxStatistics { .into_iter() .unzip(); - Self::new( - &min_max_sort_order, - &min_max_schema, - RecordBatch::try_new(Arc::clone(&min_max_schema), min_values).map_err( - |e| { - DataFusionError::ArrowError( - Box::new(e), - Some("\ncreate min batch".to_string()), - ) - }, - )?, - RecordBatch::try_new(Arc::clone(&min_max_schema), max_values).map_err( - |e| { - DataFusionError::ArrowError( - Box::new(e), - Some("\ncreate max batch".to_string()), - ) - }, - )?, - ) + let min_batch = RecordBatch::try_new(Arc::clone(&min_max_schema), min_values) + .map_err(|e| { + DataFusionError::ArrowError( + Box::new(e), + Some("\ncreate min batch".to_string()), + ) + })?; + let max_batch = RecordBatch::try_new(Arc::clone(&min_max_schema), max_values) + .map_err(|e| { + DataFusionError::ArrowError( + Box::new(e), + Some("\ncreate max batch".to_string()), + ) + })?; + + Self::new(&min_max_sort_order, &min_max_schema, min_batch, max_batch) } + #[expect(clippy::needless_pass_by_value)] pub fn new( sort_order: &LexOrdering, schema: &SchemaRef, @@ -421,6 +418,7 @@ pub async fn get_statistics_with_limit( /// /// # Returns /// A new file group with summary statistics attached +#[expect(clippy::needless_pass_by_value)] pub fn compute_file_group_statistics( file_group: FileGroup, file_schema: SchemaRef, @@ -456,6 +454,7 @@ pub fn compute_file_group_statistics( /// A tuple containing: /// * The processed file groups with their individual statistics attached /// * The summary statistics across all file groups, aka all files summary statistics +#[expect(clippy::needless_pass_by_value)] pub fn compute_all_files_statistics( file_groups: Vec, table_schema: SchemaRef, diff --git a/datafusion/datasource/src/write/demux.rs b/datafusion/datasource/src/write/demux.rs index 52cb17c10453..3fe6149b58b2 100644 --- a/datafusion/datasource/src/write/demux.rs +++ b/datafusion/datasource/src/write/demux.rs @@ -296,7 +296,7 @@ async fn hive_style_partitions_demuxer( let all_partition_values = compute_partition_keys_by_row(&rb, &partition_by)?; // Next compute how the batch should be split up to take each distinct key to its own batch - let take_map = compute_take_arrays(&rb, all_partition_values); + let take_map = compute_take_arrays(&rb, &all_partition_values); // Divide up the batch into distinct partition key batches and send each batch for (part_key, mut builder) in take_map.into_iter() { @@ -516,7 +516,7 @@ fn compute_partition_keys_by_row<'a>( fn compute_take_arrays( rb: &RecordBatch, - all_partition_values: Vec>>, + all_partition_values: &[Vec>], ) -> HashMap, UInt64Builder> { let mut take_map = HashMap::new(); for i in 0..rb.num_rows() { From 13adf2a03ce733970f077967de9b156dc24905f2 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Fri, 14 Nov 2025 10:09:33 -0800 Subject: [PATCH 11/25] feat: Handle edge case with `corr` with single row and `NaN` (#18677) ## Which issue does this PR close? - Closes https://github.com/apache/datafusion/issues/18659. ## Rationale for this change Fix an edge case in `corr` and `NaN` ## What changes are included in this PR? ## Are these changes tested? ## Are there any user-facing changes? --- .../functions-aggregate/src/correlation.rs | 137 ++++++++++-------- .../sqllogictest/test_files/aggregate.slt | 64 ++++++++ 2 files changed, 142 insertions(+), 59 deletions(-) diff --git a/datafusion/functions-aggregate/src/correlation.rs b/datafusion/functions-aggregate/src/correlation.rs index f2a464de4155..4cac159ebb3a 100644 --- a/datafusion/functions-aggregate/src/correlation.rs +++ b/datafusion/functions-aggregate/src/correlation.rs @@ -196,15 +196,24 @@ impl Accumulator for CorrelationAccumulator { } fn evaluate(&mut self) -> Result { - let n = self.covar.get_count(); - if n < 2 { - return Ok(ScalarValue::Float64(None)); - } - let covar = self.covar.evaluate()?; let stddev1 = self.stddev1.evaluate()?; let stddev2 = self.stddev2.evaluate()?; + // First check if we have NaN values by examining the internal state + // This handles the case where both inputs are NaN even with count=1 + let mean1 = self.covar.get_mean1(); + let mean2 = self.covar.get_mean2(); + + // If both means are NaN, then both input columns contain only NaN values + if mean1.is_nan() && mean2.is_nan() { + return Ok(ScalarValue::Float64(Some(f64::NAN))); + } + let n = self.covar.get_count(); + if mean1.is_nan() || mean2.is_nan() || n < 2 { + return Ok(ScalarValue::Float64(None)); + } + if let ScalarValue::Float64(Some(c)) = covar { if let ScalarValue::Float64(Some(s1)) = stddev1 { if let ScalarValue::Float64(Some(s2)) = stddev2 { @@ -402,54 +411,6 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { Ok(()) } - fn merge_batch( - &mut self, - values: &[ArrayRef], - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - ) -> Result<()> { - // Resize vectors to accommodate total number of groups - self.count.resize(total_num_groups, 0); - self.sum_x.resize(total_num_groups, 0.0); - self.sum_y.resize(total_num_groups, 0.0); - self.sum_xy.resize(total_num_groups, 0.0); - self.sum_xx.resize(total_num_groups, 0.0); - self.sum_yy.resize(total_num_groups, 0.0); - - // Extract arrays from input values - let partial_counts = values[0].as_primitive::(); - let partial_sum_x = values[1].as_primitive::(); - let partial_sum_y = values[2].as_primitive::(); - let partial_sum_xy = values[3].as_primitive::(); - let partial_sum_xx = values[4].as_primitive::(); - let partial_sum_yy = values[5].as_primitive::(); - - assert!(opt_filter.is_none(), "aggregate filter should be applied in partial stage, there should be no filter in final stage"); - - accumulate_correlation_states( - group_indices, - ( - partial_counts, - partial_sum_x, - partial_sum_y, - partial_sum_xy, - partial_sum_xx, - partial_sum_yy, - ), - |group_index, count, values| { - self.count[group_index] += count; - self.sum_x[group_index] += values[0]; - self.sum_y[group_index] += values[1]; - self.sum_xy[group_index] += values[2]; - self.sum_xx[group_index] += values[3]; - self.sum_yy[group_index] += values[4]; - }, - ); - - Ok(()) - } - fn evaluate(&mut self, emit_to: EmitTo) -> Result { let n = match emit_to { EmitTo::All => self.count.len(), @@ -465,14 +426,10 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { // - Correlation can't be calculated when a group only has 1 record, or when // the `denominator` state is 0. In these cases, the final aggregation // result should be `Null` (according to PostgreSQL's behavior). + // - However, if any of the accumulated values contain NaN, the result should + // be NaN regardless of the count (even for single-row groups). // for i in 0..n { - if self.count[i] < 2 { - values.push(0.0); - nulls.append_null(); - continue; - } - let count = self.count[i]; let sum_x = self.sum_x[i]; let sum_y = self.sum_y[i]; @@ -480,6 +437,20 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { let sum_xx = self.sum_xx[i]; let sum_yy = self.sum_yy[i]; + // If BOTH sum_x AND sum_y are NaN, then both input values are NaN → return NaN + // If only ONE of them is NaN, then only one input value is NaN → return NULL + if sum_x.is_nan() && sum_y.is_nan() { + // Both inputs are NaN → return NaN + values.push(f64::NAN); + nulls.append_non_null(); + continue; + } else if count < 2 || sum_x.is_nan() || sum_y.is_nan() { + // Only one input is NaN → return NULL + values.push(0.0); + nulls.append_null(); + continue; + } + let mean_x = sum_x / count as f64; let mean_y = sum_y / count as f64; @@ -515,6 +486,54 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { ]) } + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + // Resize vectors to accommodate total number of groups + self.count.resize(total_num_groups, 0); + self.sum_x.resize(total_num_groups, 0.0); + self.sum_y.resize(total_num_groups, 0.0); + self.sum_xy.resize(total_num_groups, 0.0); + self.sum_xx.resize(total_num_groups, 0.0); + self.sum_yy.resize(total_num_groups, 0.0); + + // Extract arrays from input values + let partial_counts = values[0].as_primitive::(); + let partial_sum_x = values[1].as_primitive::(); + let partial_sum_y = values[2].as_primitive::(); + let partial_sum_xy = values[3].as_primitive::(); + let partial_sum_xx = values[4].as_primitive::(); + let partial_sum_yy = values[5].as_primitive::(); + + assert!(opt_filter.is_none(), "aggregate filter should be applied in partial stage, there should be no filter in final stage"); + + accumulate_correlation_states( + group_indices, + ( + partial_counts, + partial_sum_x, + partial_sum_y, + partial_sum_xy, + partial_sum_xx, + partial_sum_yy, + ), + |group_index, count, values| { + self.count[group_index] += count; + self.sum_x[group_index] += values[0]; + self.sum_y[group_index] += values[1]; + self.sum_xy[group_index] += values[2]; + self.sum_xx[group_index] += values[3]; + self.sum_yy[group_index] += values[4]; + }, + ); + + Ok(()) + } + fn size(&self) -> usize { size_of_val(&self.count) + size_of_val(&self.sum_x) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 8dce114ab55f..e81bfb72a0ef 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -607,6 +607,70 @@ from data ---- 1 +# group correlation_query_with_nans_f32 +query IR +select id, corr(f, b) +from values + (1, 1, 'nan'::float), + (2, 'nan'::float, 1), + (3, 'nan'::float, null), + (4, null, 'nan'::float), + (5, 'nan'::float, 'nan'::float), + (5, 1, 1), + (5, 2, 2), + (6, 'nan'::float, 'nan'::float) t(id, f, b) +group by id +order by id +---- +1 NULL +2 NULL +3 NULL +4 NULL +5 NaN +6 NaN + +# correlation_query_with_nans_f32 +query RR +with data as ( + select 'nan'::float as f, 'nan'::float as b +) +select corr(f, b), corr('nan'::float, 'nan'::float) +from data +---- +NaN NaN + +# group correlation_query_with_nans_f64 +query IR +select id, corr(f, b) +from values + (1, 1, 'nan'::double), + (2, 'nan'::double, 1), + (3, 'nan'::double, null), + (4, null, 'nan'::float), + (5, 'nan'::double, 'nan'::double), + (5, 1, 1), + (5, 2, 2), + (6, 'nan'::double, 'nan'::double) t(id, f, b) +group by id +order by id +---- +1 NULL +2 NULL +3 NULL +4 NULL +5 NaN +6 NaN + +# correlation_query_with_nans_f64 +query RR +with data as ( + select 'nan'::double as f, 'nan'::double as b +) +select corr(f, b), corr('nan'::double, 'nan'::double) +from data +---- +NaN NaN + # csv_query_variance_1 query R SELECT var_pop(c2) FROM aggregate_test_100 From ac86c204396f2925f3ce09c06865a23d1158fbc4 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 14 Nov 2025 15:41:10 -0500 Subject: [PATCH 12/25] [main] Update changelog for 51.0.0 RC2 (#18710) ## Which issue does this PR close? - part of https://github.com/apache/datafusion/issues/17558#issuecomment-3531704481 - ## Rationale for this change Forward port changes from branch-51 to main: - https://github.com/apache/datafusion/pull/18705 ## What changes are included in this PR? Bring changes from this PR to main - https://github.com/apache/datafusion/pull/18705 ## Are these changes tested? ## Are there any user-facing changes? --- dev/changelog/51.0.0.md | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/dev/changelog/51.0.0.md b/dev/changelog/51.0.0.md index 92f0bffb1d7d..60dd24cde559 100644 --- a/dev/changelog/51.0.0.md +++ b/dev/changelog/51.0.0.md @@ -19,7 +19,7 @@ under the License. # Apache DataFusion 51.0.0 Changelog -This release consists of 533 commits from 128 contributors. See credits at the end of this changelog for more information. +This release consists of 537 commits from 129 contributors. See credits at the end of this changelog for more information. See the [upgrade guide](https://datafusion.apache.org/library-user-guide/upgrading.html) for information on how to upgrade from previous versions. @@ -573,6 +573,10 @@ See the [upgrade guide](https://datafusion.apache.org/library-user-guide/upgradi - Add comments to Cargo.toml about workspace overrides [#18526](https://github.com/apache/datafusion/pull/18526) (alamb) - minor: Remove inconsistent comment [#18539](https://github.com/apache/datafusion/pull/18539) (2010YOUY01) - Refactor `log()` signature to use coercion API + fixes [#18519](https://github.com/apache/datafusion/pull/18519) (Jefffrey) +- [branch-51] Update Changelog [#18592](https://github.com/apache/datafusion/pull/18592) (alamb) +- [branch-51] bugfix: correct regression on TableType in into_view in DF51 [#18618](https://github.com/apache/datafusion/pull/18618) (timsaucer) +- [branch-51]: Add timezone to date_trunc fast path (#18596) [#18629](https://github.com/apache/datafusion/pull/18629) (hareshkh) +- [branch-51] bugfix: select_columns should validate column names [#18624](https://github.com/apache/datafusion/pull/18624) (timsaucer) ## Credits @@ -581,7 +585,7 @@ Thank you to everyone who contributed to this release. Here is a breakdown of co ``` 88 dependabot[bot] 49 Jeffrey Vo - 34 Andrew Lamb + 35 Andrew Lamb 20 Yongting You 19 Adrian Garcia Badaracco 14 Blake Orth @@ -599,10 +603,10 @@ Thank you to everyone who contributed to this release. Here is a breakdown of co 8 Raz Luvaton 7 Adam Gutglick 7 Rohan Krishnaswamy + 7 Tim Saucer 7 kosiew 6 xudong.w 5 Nuno Faria - 5 Tim Saucer 4 Dhanush 4 Samuele Resca 4 Simon Vandel Sillesen @@ -661,6 +665,7 @@ Thank you to everyone who contributed to this release. Here is a breakdown of co 1 Gabriel 1 Gene Bordegaray 1 Georgi Krastev + 1 Haresh Khanna 1 Heran Lin 1 Hiroaki Yutani 1 Ian Lai From 8ce7acaf623ed51d4ff282efe75302731c474b03 Mon Sep 17 00:00:00 2001 From: Jeffrey Vo Date: Sat, 15 Nov 2025 13:23:16 +1100 Subject: [PATCH 13/25] Refactor Spark crc32/sha1 signatures (#18662) ## Which issue does this PR close? Part of https://github.com/apache/datafusion/issues/12725 ## Rationale for this change Prefer to avoid user_defined for consistency in function definitions. ## What changes are included in this PR? Refactor signatures of crc32 & sha1 to use coercion API instead of being user defined. Also add support for FixedSizeBinary inputs to these functions. ## Are these changes tested? Existing tests ## Are there any user-facing changes? No. --- datafusion/spark/src/function/hash/crc32.rs | 56 ++++++++---------- datafusion/spark/src/function/hash/sha1.rs | 57 ++++++++----------- .../test_files/spark/hash/crc32.slt | 31 ++++++---- .../test_files/spark/hash/sha1.slt | 25 +++++--- 4 files changed, 85 insertions(+), 84 deletions(-) diff --git a/datafusion/spark/src/function/hash/crc32.rs b/datafusion/spark/src/function/hash/crc32.rs index 76e31d12c648..8280e24b8ef5 100644 --- a/datafusion/spark/src/function/hash/crc32.rs +++ b/datafusion/spark/src/function/hash/crc32.rs @@ -22,11 +22,15 @@ use arrow::array::{ArrayRef, Int64Array}; use arrow::datatypes::DataType; use crc32fast::Hasher; use datafusion_common::cast::{ - as_binary_array, as_binary_view_array, as_large_binary_array, + as_binary_array, as_binary_view_array, as_fixed_size_binary_array, + as_large_binary_array, }; -use datafusion_common::{exec_err, internal_err, Result}; +use datafusion_common::types::{logical_string, NativeType}; +use datafusion_common::utils::take_function_args; +use datafusion_common::{internal_err, Result}; use datafusion_expr::{ - ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, + Coercion, ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, + TypeSignatureClass, Volatility, }; use datafusion_functions::utils::make_scalar_function; @@ -45,7 +49,14 @@ impl Default for SparkCrc32 { impl SparkCrc32 { pub fn new() -> Self { Self { - signature: Signature::user_defined(Volatility::Immutable), + signature: Signature::coercible( + vec![Coercion::new_implicit( + TypeSignatureClass::Binary, + vec![TypeSignatureClass::Native(logical_string())], + NativeType::Binary, + )], + Volatility::Immutable, + ), } } } @@ -70,24 +81,6 @@ impl ScalarUDFImpl for SparkCrc32 { fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { make_scalar_function(spark_crc32, vec![])(&args.args) } - - fn coerce_types(&self, arg_types: &[DataType]) -> Result> { - if arg_types.len() != 1 { - return exec_err!( - "`crc32` function requires 1 argument, got {}", - arg_types.len() - ); - } - match arg_types[0] { - DataType::Binary | DataType::LargeBinary | DataType::BinaryView => { - Ok(vec![arg_types[0].clone()]) - } - DataType::Utf8 | DataType::Utf8View => Ok(vec![DataType::Binary]), - DataType::LargeUtf8 => Ok(vec![DataType::LargeBinary]), - DataType::Null => Ok(vec![DataType::Binary]), - _ => exec_err!("`crc32` function does not support type {}", arg_types[0]), - } - } } fn spark_crc32_digest(value: &[u8]) -> i64 { @@ -104,14 +97,10 @@ fn spark_crc32_impl<'a>(input: impl Iterator>) -> ArrayR } fn spark_crc32(args: &[ArrayRef]) -> Result { - let [input] = args else { - return internal_err!( - "Spark `crc32` function requires 1 argument, got {}", - args.len() - ); - }; + let [input] = take_function_args("crc32", args)?; match input.data_type() { + DataType::Null => Ok(Arc::new(Int64Array::new_null(input.len()))), DataType::Binary => { let input = as_binary_array(input)?; Ok(spark_crc32_impl(input.iter())) @@ -124,11 +113,12 @@ fn spark_crc32(args: &[ArrayRef]) -> Result { let input = as_binary_view_array(input)?; Ok(spark_crc32_impl(input.iter())) } - _ => { - exec_err!( - "Spark `crc32` function: argument must be binary or large binary, got {:?}", - input.data_type() - ) + DataType::FixedSizeBinary(_) => { + let input = as_fixed_size_binary_array(input)?; + Ok(spark_crc32_impl(input.iter())) + } + dt => { + internal_err!("Unsupported data type for crc32: {dt}") } } } diff --git a/datafusion/spark/src/function/hash/sha1.rs b/datafusion/spark/src/function/hash/sha1.rs index 5b2a2653ed7c..9e3d96b8031a 100644 --- a/datafusion/spark/src/function/hash/sha1.rs +++ b/datafusion/spark/src/function/hash/sha1.rs @@ -22,11 +22,15 @@ use std::sync::Arc; use arrow::array::{ArrayRef, StringArray}; use arrow::datatypes::DataType; use datafusion_common::cast::{ - as_binary_array, as_binary_view_array, as_large_binary_array, + as_binary_array, as_binary_view_array, as_fixed_size_binary_array, + as_large_binary_array, }; -use datafusion_common::{exec_err, internal_err, Result}; +use datafusion_common::types::{logical_string, NativeType}; +use datafusion_common::utils::take_function_args; +use datafusion_common::{internal_err, Result}; use datafusion_expr::{ - ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, + Coercion, ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, + TypeSignatureClass, Volatility, }; use datafusion_functions::utils::make_scalar_function; use sha1::{Digest, Sha1}; @@ -47,7 +51,14 @@ impl Default for SparkSha1 { impl SparkSha1 { pub fn new() -> Self { Self { - signature: Signature::user_defined(Volatility::Immutable), + signature: Signature::coercible( + vec![Coercion::new_implicit( + TypeSignatureClass::Binary, + vec![TypeSignatureClass::Native(logical_string())], + NativeType::Binary, + )], + Volatility::Immutable, + ), aliases: vec!["sha".to_string()], } } @@ -77,24 +88,6 @@ impl ScalarUDFImpl for SparkSha1 { fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { make_scalar_function(spark_sha1, vec![])(&args.args) } - - fn coerce_types(&self, arg_types: &[DataType]) -> Result> { - if arg_types.len() != 1 { - return exec_err!( - "`sha1` function requires 1 argument, got {}", - arg_types.len() - ); - } - match arg_types[0] { - DataType::Binary | DataType::LargeBinary | DataType::BinaryView => { - Ok(vec![arg_types[0].clone()]) - } - DataType::Utf8 | DataType::Utf8View => Ok(vec![DataType::Binary]), - DataType::LargeUtf8 => Ok(vec![DataType::LargeBinary]), - DataType::Null => Ok(vec![DataType::Binary]), - _ => exec_err!("`sha1` function does not support type {}", arg_types[0]), - } - } } fn spark_sha1_digest(value: &[u8]) -> String { @@ -102,7 +95,6 @@ fn spark_sha1_digest(value: &[u8]) -> String { let mut s = String::with_capacity(result.len() * 2); #[allow(deprecated)] for b in result.as_slice() { - #[allow(clippy::unwrap_used)] write!(&mut s, "{b:02x}").unwrap(); } s @@ -116,14 +108,10 @@ fn spark_sha1_impl<'a>(input: impl Iterator>) -> ArrayRe } fn spark_sha1(args: &[ArrayRef]) -> Result { - let [input] = args else { - return internal_err!( - "Spark `sha1` function requires 1 argument, got {}", - args.len() - ); - }; + let [input] = take_function_args("sha1", args)?; match input.data_type() { + DataType::Null => Ok(Arc::new(StringArray::new_null(input.len()))), DataType::Binary => { let input = as_binary_array(input)?; Ok(spark_sha1_impl(input.iter())) @@ -136,11 +124,12 @@ fn spark_sha1(args: &[ArrayRef]) -> Result { let input = as_binary_view_array(input)?; Ok(spark_sha1_impl(input.iter())) } - _ => { - exec_err!( - "Spark `sha1` function: argument must be binary or large binary, got {:?}", - input.data_type() - ) + DataType::FixedSizeBinary(_) => { + let input = as_fixed_size_binary_array(input)?; + Ok(spark_sha1_impl(input.iter())) + } + dt => { + internal_err!("Unsupported data type for sha1: {dt}") } } } diff --git a/datafusion/sqllogictest/test_files/spark/hash/crc32.slt b/datafusion/sqllogictest/test_files/spark/hash/crc32.slt index 87b69d8d404e..6fbeb11fb9a3 100644 --- a/datafusion/sqllogictest/test_files/spark/hash/crc32.slt +++ b/datafusion/sqllogictest/test_files/spark/hash/crc32.slt @@ -30,11 +30,6 @@ SELECT crc32('Spark'); ---- 1557323817 -query I -SELECT crc32(NULL); ----- -NULL - query I SELECT crc32(''); ---- @@ -45,32 +40,48 @@ SELECT crc32(arrow_cast('', 'Binary')); ---- 0 -# Test with LargeUtf8 (using CAST to ensure type) +# Test with different types +query I +SELECT crc32(NULL); +---- +NULL + query I SELECT crc32(arrow_cast('Spark', 'LargeUtf8')); ---- 1557323817 -# Test with Utf8View (using CAST to ensure type) query I SELECT crc32(arrow_cast('Spark', 'Utf8View')); ---- 1557323817 -# Test with different binary types +query I +SELECT crc32(arrow_cast('Spark', 'Utf8')); +---- +1557323817 + query I SELECT crc32(arrow_cast('Spark', 'Binary')); ---- 1557323817 -# Test with LargeBinary +query I +SELECT crc32(arrow_cast(arrow_cast('Spark', 'Binary'), 'FixedSizeBinary(5)')); +---- +1557323817 + query I SELECT crc32(arrow_cast('Spark', 'LargeBinary')); ---- 1557323817 -# Test with BinaryView query I SELECT crc32(arrow_cast('Spark', 'BinaryView')); ---- 1557323817 + +# Upstream arrow-rs issue: https://github.com/apache/arrow-rs/issues/8841 +# This should succeed after we receive the fix +query error Arrow error: Compute error: Internal Error: Cannot cast BinaryView to BinaryArray of expected type +select crc32(arrow_cast(null, 'Dictionary(Int32, Utf8)')) diff --git a/datafusion/sqllogictest/test_files/spark/hash/sha1.slt b/datafusion/sqllogictest/test_files/spark/hash/sha1.slt index 1ce734616072..5185c45d090b 100644 --- a/datafusion/sqllogictest/test_files/spark/hash/sha1.slt +++ b/datafusion/sqllogictest/test_files/spark/hash/sha1.slt @@ -31,40 +31,51 @@ SELECT sha1('Spark'); 85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c query T -SELECT sha1(NULL); +SELECT sha('Spark'); ---- -NULL +85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c query T SELECT sha1(''); ---- da39a3ee5e6b4b0d3255bfef95601890afd80709 -# Test with LargeUtf8 (using CAST to ensure type) +# Test with different types +query T +SELECT sha1(NULL); +---- +NULL + query T SELECT sha1(arrow_cast('Spark', 'LargeUtf8')); ---- 85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c -# Test with Utf8View (using CAST to ensure type) query T SELECT sha1(arrow_cast('Spark', 'Utf8View')); ---- 85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c -# Test with Binary +query T +SELECT sha1(arrow_cast('Spark', 'Utf8')); +---- +85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c + query T SELECT sha1(arrow_cast('Spark', 'Binary')); ---- 85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c -# Test with LargeBinary +query T +SELECT sha1(arrow_cast(arrow_cast('Spark', 'Binary'), 'FixedSizeBinary(5)')); +---- +85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c + query T SELECT sha1(arrow_cast('Spark', 'LargeBinary')); ---- 85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c -# Test with BinaryView query T SELECT sha1(arrow_cast('Spark', 'BinaryView')); ---- From aaabd1fdcc55828114c7489a4572b29c605342e4 Mon Sep 17 00:00:00 2001 From: Vrishabh Date: Sat, 15 Nov 2025 08:02:42 +0530 Subject: [PATCH 14/25] feat: support spark csc (#18642) ## Which issue does this PR close? Partially implements #15914 ## Rationale for this change Spark has support for cosecant https://spark.apache.org/docs/latest/api/sql/index.html#csc . This function is not there othere in DB's like postgres, mysql and sqlite3. Hence I have added this function into datafusion-spark ## What changes are included in this PR? Adds support for cosecant in datafusion-spark ## Are these changes tested? Unit tests with inputs/outputs obtained from spark ## Are there any user-facing changes? Yes --- datafusion/spark/src/function/math/mod.rs | 4 + .../spark/src/function/math/trigonometry.rs | 97 +++++++++++++++++++ .../test_files/spark/math/csc.slt | 23 ++++- 3 files changed, 122 insertions(+), 2 deletions(-) create mode 100644 datafusion/spark/src/function/math/trigonometry.rs diff --git a/datafusion/spark/src/function/math/mod.rs b/datafusion/spark/src/function/math/mod.rs index 092335e4aa18..4560a8e72ecc 100644 --- a/datafusion/spark/src/function/math/mod.rs +++ b/datafusion/spark/src/function/math/mod.rs @@ -20,6 +20,7 @@ pub mod factorial; pub mod hex; pub mod modulus; pub mod rint; +pub mod trigonometry; pub mod width_bucket; use datafusion_expr::ScalarUDF; @@ -33,6 +34,7 @@ make_udf_function!(modulus::SparkMod, modulus); make_udf_function!(modulus::SparkPmod, pmod); make_udf_function!(rint::SparkRint, rint); make_udf_function!(width_bucket::SparkWidthBucket, width_bucket); +make_udf_function!(trigonometry::SparkCsc, csc); pub mod expr_fn { use datafusion_functions::export_functions; @@ -48,6 +50,7 @@ pub mod expr_fn { export_functions!((pmod, "Returns the positive remainder of division of the first argument by the second argument.", arg1 arg2)); export_functions!((rint, "Returns the double value that is closest in value to the argument and is equal to a mathematical integer.", arg1)); export_functions!((width_bucket, "Returns the bucket number into which the value of this expression would fall after being evaluated.", arg1 arg2 arg3 arg4)); + export_functions!((csc, "Returns the cosecant of expr.", arg1)); } pub fn functions() -> Vec> { @@ -59,5 +62,6 @@ pub fn functions() -> Vec> { pmod(), rint(), width_bucket(), + csc(), ] } diff --git a/datafusion/spark/src/function/math/trigonometry.rs b/datafusion/spark/src/function/math/trigonometry.rs new file mode 100644 index 000000000000..a6bcf8044f5a --- /dev/null +++ b/datafusion/spark/src/function/math/trigonometry.rs @@ -0,0 +1,97 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::function::error_utils::unsupported_data_type_exec_err; +use arrow::array::{ArrayRef, AsArray}; +use arrow::datatypes::{DataType, Float64Type}; +use datafusion_common::utils::take_function_args; +use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::{ + ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, +}; +use std::any::Any; +use std::sync::Arc; + +static CSC_FUNCTION_NAME: &str = "csc"; + +/// +#[derive(Debug, PartialEq, Eq, Hash)] +pub struct SparkCsc { + signature: Signature, +} + +impl Default for SparkCsc { + fn default() -> Self { + Self::new() + } +} + +impl SparkCsc { + pub fn new() -> Self { + Self { + signature: Signature::exact(vec![DataType::Float64], Volatility::Immutable), + } + } +} + +impl ScalarUDFImpl for SparkCsc { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + CSC_FUNCTION_NAME + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Float64) + } + + fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + let [arg] = take_function_args(self.name(), &args.args)?; + spark_csc(arg) + } +} + +fn spark_csc(arg: &ColumnarValue) -> Result { + match arg { + ColumnarValue::Scalar(ScalarValue::Float64(value)) => Ok(ColumnarValue::Scalar( + ScalarValue::Float64(value.map(|x| 1.0 / x.sin())), + )), + ColumnarValue::Array(array) => match array.data_type() { + DataType::Float64 => Ok(ColumnarValue::Array(Arc::new( + array + .as_primitive::() + .unary::<_, Float64Type>(|x| 1.0 / x.sin()), + ) as ArrayRef)), + other => Err(unsupported_data_type_exec_err( + CSC_FUNCTION_NAME, + format!("{}", DataType::Float64).as_str(), + other, + )), + }, + other => Err(unsupported_data_type_exec_err( + CSC_FUNCTION_NAME, + format!("{}", DataType::Float64).as_str(), + &other.data_type(), + )), + } +} diff --git a/datafusion/sqllogictest/test_files/spark/math/csc.slt b/datafusion/sqllogictest/test_files/spark/math/csc.slt index b11986c3e1b9..5eb9f4447280 100644 --- a/datafusion/sqllogictest/test_files/spark/math/csc.slt +++ b/datafusion/sqllogictest/test_files/spark/math/csc.slt @@ -23,5 +23,24 @@ ## Original Query: SELECT csc(1); ## PySpark 3.5.5 Result: {'CSC(1)': 1.1883951057781212, 'typeof(CSC(1))': 'double', 'typeof(1)': 'int'} -#query -#SELECT csc(1::int); + +query R +SELECT csc(1::INT); +---- +1.188395105778121 + +query R +SELECT csc(a) FROM (VALUES (0::INT), (1::INT), (-1::INT), (null)) AS t(a); +---- +Infinity +1.188395105778121 +-1.188395105778121 +NULL + +query R +SELECT csc(a) FROM (VALUES (pi()), (-pi()), (pi()/2) , (arrow_cast('NAN','Float32'))) AS t(a); +---- +8165619676597685 +-8165619676597685 +1 +NaN \ No newline at end of file From d61021cc9cb895e2707786b5b752e647b7d2ed0a Mon Sep 17 00:00:00 2001 From: Jeffrey Vo Date: Sat, 15 Nov 2025 13:41:37 +1100 Subject: [PATCH 15/25] CI: try free up space in `Rust / cargo test (amd64)` action (#18709) Closes #18692 (hopefully) Trying to get CI to pass consistently, try various techniques. --- .github/workflows/rust.yml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index c58f888df163..125273448df4 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -271,7 +271,21 @@ jobs: runs-on: ubuntu-latest container: image: amd64/rust + volumes: + - /usr/local:/host/usr/local steps: + - name: Remove unnecessary preinstalled software + run: | + echo "Disk space before cleanup:" + df -h + # remove tool cache: about 8.5GB (github has host /opt/hostedtoolcache mounted as /__t) + rm -rf /__t/* || true + # remove Haskell runtime: about 6.3GB (host /usr/local/.ghcup) + rm -rf /host/usr/local/.ghcup || true + # remove Android library: about 7.8GB (host /usr/local/lib/android) + rm -rf /host/usr/local/lib/android || true + echo "Disk space after cleanup:" + df -h - uses: actions/checkout@08c6903cd8c0fde910a37f88322edcfb5dd907a8 # v5.0.0 with: submodules: true From a1c04c9ba93e57f6e66f3fd5499d3d4b43f9fdaa Mon Sep 17 00:00:00 2001 From: Dhanush Date: Sat, 15 Nov 2025 10:09:53 +0530 Subject: [PATCH 16/25] chore: enforce clippy lint needless_pass_by_value to datafusion-proto (#18715) ## Which issue does this PR close? - Part of parent issue #18503 ## What changes are included in this PR? enforce clippy lint `needless_pass_by_value` to `datafusion-proto` ## Are these changes tested? yes ## Are there any user-facing changes? no --- datafusion/proto/src/lib.rs | 3 +++ datafusion/proto/src/physical_plan/to_proto.rs | 1 + 2 files changed, 4 insertions(+) diff --git a/datafusion/proto/src/lib.rs b/datafusion/proto/src/lib.rs index b16b12bc0516..130538d5af9f 100644 --- a/datafusion/proto/src/lib.rs +++ b/datafusion/proto/src/lib.rs @@ -23,6 +23,9 @@ // Make sure fast / cheap clones on Arc are explicit: // https://github.com/apache/datafusion/issues/11143 #![deny(clippy::clone_on_ref_ptr)] +// https://github.com/apache/datafusion/issues/18503 +#![deny(clippy::needless_pass_by_value)] +#![cfg_attr(test, allow(clippy::needless_pass_by_value))] //! Serialize / Deserialize DataFusion Plans to bytes //! diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index dc0a78dbccf1..820628ff70cd 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -52,6 +52,7 @@ use crate::protobuf::{ use super::PhysicalExtensionCodec; +#[expect(clippy::needless_pass_by_value)] pub fn serialize_physical_aggr_expr( aggr_expr: Arc, codec: &dyn PhysicalExtensionCodec, From 19219190ddd7a74a58ef41b1b835bc56eac26ee6 Mon Sep 17 00:00:00 2001 From: Dhanush Date: Sat, 15 Nov 2025 10:11:36 +0530 Subject: [PATCH 17/25] chore: enforce clippy lint needless_pass_by_value to datafusion-spark (#18714) ## Which issue does this PR close? - Part of parent issue #18503 ## What changes are included in this PR? enforce clippy lint `needless_pass_by_value` to `datafusion-spark` ## Are these changes tested? yes ## Are there any user-facing changes? no --- .../src/function/string/format_string.rs | 25 ++++++++----------- .../spark/src/function/string/length.rs | 16 ++++++------ datafusion/spark/src/lib.rs | 3 +++ 3 files changed, 21 insertions(+), 23 deletions(-) diff --git a/datafusion/spark/src/function/string/format_string.rs b/datafusion/spark/src/function/string/format_string.rs index 9809456af9a4..adb0eb2f0995 100644 --- a/datafusion/spark/src/function/string/format_string.rs +++ b/datafusion/spark/src/function/string/format_string.rs @@ -304,7 +304,7 @@ impl<'a> Formatter<'a> { return exec_err!("No previous argument to reference"); }; let (spec, rest) = - take_conversion_specifier(rest, p, arg_types[p - 1].clone())?; + take_conversion_specifier(rest, p, &arg_types[p - 1])?; res.push(FormatElement::Format(spec)); rem = rest; continue; @@ -335,7 +335,7 @@ impl<'a> Formatter<'a> { let (spec, rest) = take_conversion_specifier( rest, current_argument_index, - arg_types[current_argument_index - 1].clone(), + &arg_types[current_argument_index - 1], ) .map_err(|e| exec_datafusion_err!("{:?}, format string: {:?}", e, fmt))?; res.push(FormatElement::Format(spec)); @@ -582,7 +582,7 @@ impl TryFrom for TimeFormat { } impl ConversionType { - pub fn validate(&self, arg_type: DataType) -> Result<()> { + pub fn validate(&self, arg_type: &DataType) -> Result<()> { match self { ConversionType::BooleanLower | ConversionType::BooleanUpper => { if !matches!(arg_type, DataType::Boolean) { @@ -716,11 +716,11 @@ impl ConversionType { } } -fn take_conversion_specifier( - mut s: &str, +fn take_conversion_specifier<'a>( + mut s: &'a str, argument_index: usize, - arg_type: DataType, -) -> Result<(ConversionSpecifier, &str)> { + arg_type: &DataType, +) -> Result<(ConversionSpecifier, &'a str)> { let mut spec = ConversionSpecifier { argument_index, alt_form: false, @@ -1186,7 +1186,7 @@ impl ConversionSpecifier { | ConversionType::CompactFloatLower | ConversionType::CompactFloatUpper, Some(value), - ) => self.format_decimal(string, value.to_string(), *scale as i64), + ) => self.format_decimal(string, &value.to_string(), *scale as i64), ( ConversionType::StringLower | ConversionType::StringUpper, Some(value), @@ -1212,7 +1212,7 @@ impl ConversionSpecifier { | ConversionType::CompactFloatLower | ConversionType::CompactFloatUpper, Some(value), - ) => self.format_decimal(string, value.to_string(), *scale as i64), + ) => self.format_decimal(string, &value.to_string(), *scale as i64), ( ConversionType::StringLower | ConversionType::StringUpper, Some(value), @@ -1991,12 +1991,7 @@ impl ConversionSpecifier { } } - fn format_decimal( - &self, - writer: &mut String, - value: String, - scale: i64, - ) -> Result<()> { + fn format_decimal(&self, writer: &mut String, value: &str, scale: i64) -> Result<()> { let mut prefix = String::new(); let upper = self.conversion_type.is_upper(); diff --git a/datafusion/spark/src/function/string/length.rs b/datafusion/spark/src/function/string/length.rs index 1fa54d000eff..ac6030770fe2 100644 --- a/datafusion/spark/src/function/string/length.rs +++ b/datafusion/spark/src/function/string/length.rs @@ -98,33 +98,33 @@ fn spark_length(args: &[ArrayRef]) -> datafusion_common::Result { match args[0].data_type() { DataType::Utf8 => { let string_array = args[0].as_string::(); - character_length::<_>(string_array) + character_length::<_>(&string_array) } DataType::LargeUtf8 => { let string_array = args[0].as_string::(); - character_length::<_>(string_array) + character_length::<_>(&string_array) } DataType::Utf8View => { let string_array = args[0].as_string_view(); - character_length::<_>(string_array) + character_length::<_>(&string_array) } DataType::Binary => { let binary_array = args[0].as_binary::(); - byte_length::<_>(binary_array) + byte_length::<_>(&binary_array) } DataType::LargeBinary => { let binary_array = args[0].as_binary::(); - byte_length::<_>(binary_array) + byte_length::<_>(&binary_array) } DataType::BinaryView => { let binary_array = args[0].as_binary_view(); - byte_length::<_>(binary_array) + byte_length::<_>(&binary_array) } other => exec_err!("Unsupported data type {other:?} for function `length`"), } } -fn character_length<'a, V>(array: V) -> datafusion_common::Result +fn character_length<'a, V>(array: &V) -> datafusion_common::Result where V: StringArrayType<'a>, { @@ -169,7 +169,7 @@ where Ok(Arc::new(array)) } -fn byte_length<'a, V>(array: V) -> datafusion_common::Result +fn byte_length<'a, V>(array: &V) -> datafusion_common::Result where V: BinaryArrayType<'a>, { diff --git a/datafusion/spark/src/lib.rs b/datafusion/spark/src/lib.rs index 5b1fa06cb2c7..4f3f795add5e 100644 --- a/datafusion/spark/src/lib.rs +++ b/datafusion/spark/src/lib.rs @@ -22,6 +22,9 @@ #![cfg_attr(docsrs, feature(doc_cfg))] // Make cheap clones clear: https://github.com/apache/datafusion/issues/11143 #![deny(clippy::clone_on_ref_ptr)] +// https://github.com/apache/datafusion/issues/18503 +#![deny(clippy::needless_pass_by_value)] +#![cfg_attr(test, allow(clippy::needless_pass_by_value))] //! Spark Expression packages for [DataFusion]. //! From 55b80559bbbc07c866407fbf27426c66da60b04f Mon Sep 17 00:00:00 2001 From: jizezhang Date: Fri, 14 Nov 2025 22:44:01 -0800 Subject: [PATCH 18/25] pass session state to cache producer --- datafusion/core/src/dataframe/mod.rs | 5 +++-- datafusion/core/src/execution/context/mod.rs | 10 ++++++++-- datafusion/core/src/test_util/mod.rs | 8 ++++++-- 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index aaeb22f374fa..975f5c0fb25b 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2329,7 +2329,8 @@ impl DataFrame { }) } - /// Cache DataFrame as a memory table. + /// Cache DataFrame as a memory table by default, or use + /// a [`CacheProducer`] if configured via [`SessionState`]. /// /// ``` /// # use datafusion::prelude::*; @@ -2346,7 +2347,7 @@ impl DataFrame { /// ``` pub async fn cache(self) -> Result { if let Some(cache_producer) = self.session_state.cache_producer() { - let node = cache_producer.create(self.plan)?; + let node = cache_producer.create(self.plan, self.session_state.as_ref())?; let plan = LogicalPlan::Extension(Extension { node }); Ok(Self { session_state: self.session_state, diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 4bee9a22d316..13e9c53e15ac 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -1893,10 +1893,16 @@ pub enum RegisterFunction { /// Interface for applying a custom caching strategy. /// Implement this trait and register via [`SessionState`] /// to create a custom logical node for caching. +/// Additionally, a custom [`ExtensionPlanner`]/[`QueryPlanner`] +/// need to be implemented to handle plans containing such node. pub trait CacheProducer: Debug + Sync + Send { /// Create a custom logical node for caching - /// given a logical plan (of DF to cache). - fn create(&self, plan: LogicalPlan) -> Result>; + /// given a logical plan (of DF to cache) and a session state. + fn create( + &self, + plan: LogicalPlan, + session_state: &SessionState, + ) -> Result>; } /// Default implementation of [SerializerRegistry] that throws unimplemented error diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 4171096ec6cc..589e6a3fb7e2 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -42,7 +42,7 @@ use crate::logical_expr::{LogicalPlanBuilder, UNNAMED_TABLE}; use crate::physical_plan::ExecutionPlan; use crate::prelude::{CsvReadOptions, SessionContext}; -use crate::execution::SendableRecordBatchStream; +use crate::execution::{SendableRecordBatchStream, SessionState}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_catalog::Session; @@ -330,7 +330,11 @@ impl UserDefinedLogicalNodeCore for CacheNode { struct TestCacheProducer {} impl CacheProducer for TestCacheProducer { - fn create(&self, plan: LogicalPlan) -> Result> { + fn create( + &self, + plan: LogicalPlan, + _session_state: &SessionState, + ) -> Result> { Ok(Arc::new(CacheNode { input: plan })) } } From f95bc670fd6f8fb9d2899b64516b435d89ae6f51 Mon Sep 17 00:00:00 2001 From: jizezhang Date: Fri, 14 Nov 2025 23:15:25 -0800 Subject: [PATCH 19/25] fix imports in doc --- datafusion/core/src/dataframe/mod.rs | 2 +- datafusion/core/src/execution/context/mod.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 975f5c0fb25b..94ada464cb8e 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2330,7 +2330,7 @@ impl DataFrame { } /// Cache DataFrame as a memory table by default, or use - /// a [`CacheProducer`] if configured via [`SessionState`]. + /// a [`crate::execution::context::CacheProducer`] if configured via [`SessionState`]. /// /// ``` /// # use datafusion::prelude::*; diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 13e9c53e15ac..77c2a079f46b 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -1893,7 +1893,7 @@ pub enum RegisterFunction { /// Interface for applying a custom caching strategy. /// Implement this trait and register via [`SessionState`] /// to create a custom logical node for caching. -/// Additionally, a custom [`ExtensionPlanner`]/[`QueryPlanner`] +/// Additionally, a custom [`crate::physical_planner::ExtensionPlanner`]/[`QueryPlanner`] /// need to be implemented to handle plans containing such node. pub trait CacheProducer: Debug + Sync + Send { /// Create a custom logical node for caching From 213c6f635ba3772c6609156c46877f9f30f30d2f Mon Sep 17 00:00:00 2001 From: jizezhang Date: Mon, 17 Nov 2025 22:30:05 -0800 Subject: [PATCH 20/25] refactor --- datafusion/core/src/dataframe/mod.rs | 10 ++-- datafusion/core/src/execution/context/mod.rs | 21 -------- .../core/src/execution/session_state.rs | 51 ++++++++++--------- datafusion/core/src/test_util/mod.rs | 30 +++++------ 4 files changed, 45 insertions(+), 67 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 94ada464cb8e..6656f333df55 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -56,7 +56,6 @@ use datafusion_common::{ ScalarValue, SchemaError, TableReference, UnnestOptions, }; use datafusion_expr::select_expr::SelectExpr; -use datafusion_expr::Extension; use datafusion_expr::{ case, dml::InsertOp, @@ -2330,7 +2329,7 @@ impl DataFrame { } /// Cache DataFrame as a memory table by default, or use - /// a [`crate::execution::context::CacheProducer`] if configured via [`SessionState`]. + /// a [`crate::execution::session_state::CacheFactory`] if configured via [`SessionState`]. /// /// ``` /// # use datafusion::prelude::*; @@ -2346,12 +2345,11 @@ impl DataFrame { /// # } /// ``` pub async fn cache(self) -> Result { - if let Some(cache_producer) = self.session_state.cache_producer() { - let node = cache_producer.create(self.plan, self.session_state.as_ref())?; - let plan = LogicalPlan::Extension(Extension { node }); + if let Some(cache_factory) = self.session_state.cache_factory() { + let new_plan = cache_factory(self.plan, self.session_state.as_ref())?; Ok(Self { session_state: self.session_state, - plan, + plan: new_plan, projection_requires_validation: self.projection_requires_validation, }) } else { diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index aa61ff48c04d..9c7339e6748e 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -472,12 +472,6 @@ impl SessionContext { self } - /// Register a [`CacheProducer`] to provide custom caching strategy - pub fn with_cache_producer(self, cache_producer: Arc) -> Self { - self.state.write().set_cache_producer(cache_producer); - self - } - /// Adds an optimizer rule to the end of the existing rules. /// /// See [`SessionState`] for more control of when the rule is applied. @@ -1956,21 +1950,6 @@ pub enum RegisterFunction { Table(String, Arc), } -/// Interface for applying a custom caching strategy. -/// Implement this trait and register via [`SessionState`] -/// to create a custom logical node for caching. -/// Additionally, a custom [`crate::physical_planner::ExtensionPlanner`]/[`QueryPlanner`] -/// need to be implemented to handle plans containing such node. -pub trait CacheProducer: Debug + Sync + Send { - /// Create a custom logical node for caching - /// given a logical plan (of DF to cache) and a session state. - fn create( - &self, - plan: LogicalPlan, - session_state: &SessionState, - ) -> Result>; -} - /// Default implementation of [SerializerRegistry] that throws unimplemented error /// for all requests. #[derive(Debug)] diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 40709868c084..5e896fda557b 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -27,9 +27,7 @@ use crate::catalog::{CatalogProviderList, SchemaProvider, TableProviderFactory}; use crate::datasource::file_format::FileFormatFactory; #[cfg(feature = "sql")] use crate::datasource::provider_as_source; -use crate::execution::context::{ - CacheProducer, EmptySerializerRegistry, FunctionFactory, QueryPlanner, -}; +use crate::execution::context::{EmptySerializerRegistry, FunctionFactory, QueryPlanner}; use crate::execution::SessionStateDefaults; use crate::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner}; use arrow_schema::{DataType, FieldRef}; @@ -92,6 +90,14 @@ use sqlparser::{ use url::Url; use uuid::Uuid; +/// Used for supplying a custom caching strategy. +/// A [`CacheFactory`] can be registered via [`SessionState`] +/// to create a custom logical plan for caching. +/// Additionally, a custom [`crate::physical_planner::ExtensionPlanner`]/[`QueryPlanner`] +/// need to be implemented to handle such plans. +pub type CacheFactory = + fn(LogicalPlan, &SessionState) -> datafusion_common::Result; + /// `SessionState` contains all the necessary state to plan and execute queries, /// such as configuration, functions, and runtime environment. Please see the /// documentation on [`SessionContext`] for more information. @@ -187,7 +193,7 @@ pub struct SessionState { /// It will be invoked on `CREATE FUNCTION` statements. /// thus, changing dialect o PostgreSql is required function_factory: Option>, - cache_producer: Option>, + cache_factory: Option, /// Cache logical plans of prepared statements for later execution. /// Key is the prepared statement name. prepared_plans: HashMap>, @@ -209,7 +215,7 @@ impl Debug for SessionState { .field("table_options", &self.table_options) .field("table_factories", &self.table_factories) .field("function_factory", &self.function_factory) - .field("cache_producer", &self.cache_producer) + .field("cache_factory", &self.cache_factory) .field("expr_planners", &self.expr_planners); #[cfg(feature = "sql")] @@ -359,14 +365,14 @@ impl SessionState { self.function_factory.as_ref() } - /// Register a [`CacheProducer`] for custom caching strategy - pub fn set_cache_producer(&mut self, cache_producer: Arc) { - self.cache_producer = Some(cache_producer); + /// Register a [`CacheFactory`] for custom caching strategy + pub fn set_cache_factory(&mut self, cache_factory: CacheFactory) { + self.cache_factory = Some(cache_factory); } /// Get the cache producer - pub fn cache_producer(&self) -> Option<&Arc> { - self.cache_producer.as_ref() + pub fn cache_factory(&self) -> Option<&CacheFactory> { + self.cache_factory.as_ref() } /// Get the table factories @@ -955,7 +961,7 @@ pub struct SessionStateBuilder { table_factories: Option>>, runtime_env: Option>, function_factory: Option>, - cache_producer: Option>, + cache_factory: Option, // fields to support convenience functions analyzer_rules: Option>>, optimizer_rules: Option>>, @@ -993,7 +999,7 @@ impl SessionStateBuilder { table_factories: None, runtime_env: None, function_factory: None, - cache_producer: None, + cache_factory: None, // fields to support convenience functions analyzer_rules: None, optimizer_rules: None, @@ -1046,7 +1052,7 @@ impl SessionStateBuilder { table_factories: Some(existing.table_factories), runtime_env: Some(existing.runtime_env), function_factory: existing.function_factory, - cache_producer: existing.cache_producer, + cache_factory: existing.cache_factory, // fields to support convenience functions analyzer_rules: None, optimizer_rules: None, @@ -1335,12 +1341,9 @@ impl SessionStateBuilder { self } - /// Set a [`CacheProducer`] for custom caching strategy - pub fn with_cache_producer( - mut self, - cache_producer: Option>, - ) -> Self { - self.cache_producer = cache_producer; + /// Set a [`CacheFactory`] for custom caching strategy + pub fn with_cache_factory(mut self, cache_factory: Option) -> Self { + self.cache_factory = cache_factory; self } @@ -1407,7 +1410,7 @@ impl SessionStateBuilder { table_factories, runtime_env, function_factory, - cache_producer, + cache_factory, analyzer_rules, optimizer_rules, physical_optimizer_rules, @@ -1444,7 +1447,7 @@ impl SessionStateBuilder { table_factories: table_factories.unwrap_or_default(), runtime_env, function_factory, - cache_producer, + cache_factory, prepared_plans: HashMap::new(), }; @@ -1649,8 +1652,8 @@ impl SessionStateBuilder { } /// Returns the cache producer - pub fn cache_producer(&mut self) -> &mut Option> { - &mut self.cache_producer + pub fn cache_factory(&mut self) -> &mut Option { + &mut self.cache_factory } /// Returns the current analyzer_rules value @@ -1691,7 +1694,7 @@ impl Debug for SessionStateBuilder { .field("table_options", &self.table_options) .field("table_factories", &self.table_factories) .field("function_factory", &self.function_factory) - .field("cache_producer", &self.cache_producer) + .field("cache_factory", &self.cache_factory) .field("expr_planners", &self.expr_planners); #[cfg(feature = "sql")] let ret = ret.field("type_planner", &self.type_planner); diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 589e6a3fb7e2..9f16cd20f851 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -37,18 +37,17 @@ use crate::dataframe::DataFrame; use crate::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; use crate::datasource::{empty::EmptyTable, provider_as_source}; use crate::error::Result; -use crate::execution::context::CacheProducer; use crate::logical_expr::{LogicalPlanBuilder, UNNAMED_TABLE}; use crate::physical_plan::ExecutionPlan; use crate::prelude::{CsvReadOptions, SessionContext}; -use crate::execution::{SendableRecordBatchStream, SessionState}; +use crate::execution::{SendableRecordBatchStream, SessionState, SessionStateBuilder}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_catalog::Session; use datafusion_common::{DFSchemaRef, TableReference}; use datafusion_expr::{ - CreateExternalTable, Expr, LogicalPlan, SortExpr, TableType, UserDefinedLogicalNode, + CreateExternalTable, Expr, LogicalPlan, SortExpr, TableType, UserDefinedLogicalNodeCore, }; use std::pin::Pin; @@ -326,22 +325,21 @@ impl UserDefinedLogicalNodeCore for CacheNode { } } -#[derive(Debug)] -struct TestCacheProducer {} - -impl CacheProducer for TestCacheProducer { - fn create( - &self, - plan: LogicalPlan, - _session_state: &SessionState, - ) -> Result> { - Ok(Arc::new(CacheNode { input: plan })) - } +fn cache_factory( + plan: LogicalPlan, + _session_state: &SessionState, +) -> Result { + Ok(LogicalPlan::Extension(datafusion_expr::Extension { + node: Arc::new(CacheNode { input: plan }), + })) } /// Create a test table registered to a session context with an associated cache producer -pub async fn test_table_with_cache_producer() -> Result { - let ctx = SessionContext::new().with_cache_producer(Arc::new(TestCacheProducer {})); +pub async fn test_table_with_cache_factory() -> Result { + let session_state = SessionStateBuilder::new() + .with_cache_factory(Some(cache_factory)) + .build(); + let ctx = SessionContext::new_with_state(session_state); let name = "aggregate_test_100"; register_aggregate_csv(&ctx, name).await?; ctx.table(name).await From 1c67becb1c0ee0ef4294d707514738afc2e0ee83 Mon Sep 17 00:00:00 2001 From: jizezhang Date: Mon, 17 Nov 2025 23:22:03 -0800 Subject: [PATCH 21/25] update doc and comment --- datafusion/core/src/execution/session_state.rs | 7 +++---- datafusion/core/src/test_util/mod.rs | 2 +- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 5e896fda557b..6e300f80582d 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -90,11 +90,10 @@ use sqlparser::{ use url::Url; use uuid::Uuid; -/// Used for supplying a custom caching strategy. /// A [`CacheFactory`] can be registered via [`SessionState`] /// to create a custom logical plan for caching. /// Additionally, a custom [`crate::physical_planner::ExtensionPlanner`]/[`QueryPlanner`] -/// need to be implemented to handle such plans. +/// may need to be implemented to handle such plans. pub type CacheFactory = fn(LogicalPlan, &SessionState) -> datafusion_common::Result; @@ -370,7 +369,7 @@ impl SessionState { self.cache_factory = Some(cache_factory); } - /// Get the cache producer + /// Get the cache factory pub fn cache_factory(&self) -> Option<&CacheFactory> { self.cache_factory.as_ref() } @@ -1651,7 +1650,7 @@ impl SessionStateBuilder { &mut self.function_factory } - /// Returns the cache producer + /// Returns the cache factory pub fn cache_factory(&mut self) -> &mut Option { &mut self.cache_factory } diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 9f16cd20f851..eed5897e635a 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -334,7 +334,7 @@ fn cache_factory( })) } -/// Create a test table registered to a session context with an associated cache producer +/// Create a test table registered to a session context with an associated cache factory pub async fn test_table_with_cache_factory() -> Result { let session_state = SessionStateBuilder::new() .with_cache_factory(Some(cache_factory)) From 5d1c9994fd9ff17e3ee95e34a47e71570d65155a Mon Sep 17 00:00:00 2001 From: jizezhang Date: Tue, 18 Nov 2025 00:00:59 -0800 Subject: [PATCH 22/25] fix test name --- datafusion/core/tests/dataframe/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index cb66fde2976b..ec163c32f267 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -61,7 +61,7 @@ use datafusion::prelude::{ }; use datafusion::test_util::{ parquet_test_data, populate_csv_partitions, register_aggregate_csv, test_table, - test_table_with_cache_producer, test_table_with_name, + test_table_with_cache_factory, test_table_with_name, }; use datafusion_catalog::TableProvider; use datafusion_common::test_util::{batches_to_sort_string, batches_to_string}; @@ -2337,7 +2337,7 @@ async fn cache_test() -> Result<()> { #[tokio::test] async fn cache_producer_test() -> Result<()> { - let df = test_table_with_cache_producer() + let df = test_table_with_cache_factory() .await? .select_columns(&["c2", "c3"])? .limit(0, Some(1))? From e94b2062a868fb9f2e68d63b83dc58be8ffb88ac Mon Sep 17 00:00:00 2001 From: jizezhang Date: Thu, 20 Nov 2025 10:07:35 -0800 Subject: [PATCH 23/25] revert to use trait for cache factory --- datafusion/core/src/dataframe/mod.rs | 10 ++++-- .../core/src/execution/session_state.rs | 35 ++++++++++++------- datafusion/core/src/test_util/mod.rs | 23 +++++++----- 3 files changed, 44 insertions(+), 24 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 6656f333df55..a72dac4d374f 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2328,8 +2328,11 @@ impl DataFrame { }) } - /// Cache DataFrame as a memory table by default, or use - /// a [`crate::execution::session_state::CacheFactory`] if configured via [`SessionState`]. + /// Cache DataFrame as a memory table. + /// + /// Default behavior could be changed using + /// a [`crate::execution::session_state::CacheFactory`] + /// configured via [`SessionState`]. /// /// ``` /// # use datafusion::prelude::*; @@ -2346,7 +2349,8 @@ impl DataFrame { /// ``` pub async fn cache(self) -> Result { if let Some(cache_factory) = self.session_state.cache_factory() { - let new_plan = cache_factory(self.plan, self.session_state.as_ref())?; + let new_plan = + cache_factory.create(self.plan, self.session_state.as_ref())?; Ok(Self { session_state: self.session_state, plan: new_plan, diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 6e300f80582d..8d016e867e7d 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -90,13 +90,6 @@ use sqlparser::{ use url::Url; use uuid::Uuid; -/// A [`CacheFactory`] can be registered via [`SessionState`] -/// to create a custom logical plan for caching. -/// Additionally, a custom [`crate::physical_planner::ExtensionPlanner`]/[`QueryPlanner`] -/// may need to be implemented to handle such plans. -pub type CacheFactory = - fn(LogicalPlan, &SessionState) -> datafusion_common::Result; - /// `SessionState` contains all the necessary state to plan and execute queries, /// such as configuration, functions, and runtime environment. Please see the /// documentation on [`SessionContext`] for more information. @@ -192,7 +185,7 @@ pub struct SessionState { /// It will be invoked on `CREATE FUNCTION` statements. /// thus, changing dialect o PostgreSql is required function_factory: Option>, - cache_factory: Option, + cache_factory: Option>, /// Cache logical plans of prepared statements for later execution. /// Key is the prepared statement name. prepared_plans: HashMap>, @@ -365,12 +358,12 @@ impl SessionState { } /// Register a [`CacheFactory`] for custom caching strategy - pub fn set_cache_factory(&mut self, cache_factory: CacheFactory) { + pub fn set_cache_factory(&mut self, cache_factory: Arc) { self.cache_factory = Some(cache_factory); } /// Get the cache factory - pub fn cache_factory(&self) -> Option<&CacheFactory> { + pub fn cache_factory(&self) -> Option<&Arc> { self.cache_factory.as_ref() } @@ -960,7 +953,7 @@ pub struct SessionStateBuilder { table_factories: Option>>, runtime_env: Option>, function_factory: Option>, - cache_factory: Option, + cache_factory: Option>, // fields to support convenience functions analyzer_rules: Option>>, optimizer_rules: Option>>, @@ -1341,7 +1334,10 @@ impl SessionStateBuilder { } /// Set a [`CacheFactory`] for custom caching strategy - pub fn with_cache_factory(mut self, cache_factory: Option) -> Self { + pub fn with_cache_factory( + mut self, + cache_factory: Option>, + ) -> Self { self.cache_factory = cache_factory; self } @@ -1651,7 +1647,7 @@ impl SessionStateBuilder { } /// Returns the cache factory - pub fn cache_factory(&mut self) -> &mut Option { + pub fn cache_factory(&mut self) -> &mut Option> { &mut self.cache_factory } @@ -2082,6 +2078,19 @@ pub(crate) struct PreparedPlan { pub(crate) plan: Arc, } +/// A [`CacheFactory`] can be registered via [`SessionState`] +/// to create a custom logical plan for caching. +/// Additionally, a custom [`crate::physical_planner::ExtensionPlanner`]/[`QueryPlanner`] +/// may need to be implemented to handle such plans. +pub trait CacheFactory: Debug + Send + Sync { + /// Create a logical plan for caching + fn create( + &self, + plan: LogicalPlan, + session_state: &SessionState, + ) -> datafusion_common::Result; +} + #[cfg(test)] mod tests { use super::{SessionContextProvider, SessionStateBuilder}; diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index eed5897e635a..466ee38a426f 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -37,6 +37,7 @@ use crate::dataframe::DataFrame; use crate::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; use crate::datasource::{empty::EmptyTable, provider_as_source}; use crate::error::Result; +use crate::execution::session_state::CacheFactory; use crate::logical_expr::{LogicalPlanBuilder, UNNAMED_TABLE}; use crate::physical_plan::ExecutionPlan; use crate::prelude::{CsvReadOptions, SessionContext}; @@ -325,19 +326,25 @@ impl UserDefinedLogicalNodeCore for CacheNode { } } -fn cache_factory( - plan: LogicalPlan, - _session_state: &SessionState, -) -> Result { - Ok(LogicalPlan::Extension(datafusion_expr::Extension { - node: Arc::new(CacheNode { input: plan }), - })) +#[derive(Debug)] +struct TestCacheFactory {} + +impl CacheFactory for TestCacheFactory { + fn create( + &self, + plan: LogicalPlan, + _session_state: &SessionState, + ) -> Result { + Ok(LogicalPlan::Extension(datafusion_expr::Extension { + node: Arc::new(CacheNode { input: plan }), + })) + } } /// Create a test table registered to a session context with an associated cache factory pub async fn test_table_with_cache_factory() -> Result { let session_state = SessionStateBuilder::new() - .with_cache_factory(Some(cache_factory)) + .with_cache_factory(Some(Arc::new(TestCacheFactory {}))) .build(); let ctx = SessionContext::new_with_state(session_state); let name = "aggregate_test_100"; From 85ebd75f743a17ada1b3795d64a0d923d85f41f9 Mon Sep 17 00:00:00 2001 From: jizezhang Date: Thu, 20 Nov 2025 18:19:54 -0800 Subject: [PATCH 24/25] update df initialization --- datafusion/core/src/dataframe/mod.rs | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index a72dac4d374f..8e52ca1bf1af 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2351,11 +2351,7 @@ impl DataFrame { if let Some(cache_factory) = self.session_state.cache_factory() { let new_plan = cache_factory.create(self.plan, self.session_state.as_ref())?; - Ok(Self { - session_state: self.session_state, - plan: new_plan, - projection_requires_validation: self.projection_requires_validation, - }) + Ok(Self::new(*self.session_state, new_plan)) } else { let context = SessionContext::new_with_state((*self.session_state).clone()); // The schema is consistent with the output From 73937e30cb9ec63ffdc966a2705eac97be1fa1fd Mon Sep 17 00:00:00 2001 From: jizezhang Date: Sun, 23 Nov 2025 11:10:09 -0800 Subject: [PATCH 25/25] update comment for cache factory --- datafusion/core/src/execution/session_state.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 8d016e867e7d..8bc526fbfb8c 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -2079,7 +2079,7 @@ pub(crate) struct PreparedPlan { } /// A [`CacheFactory`] can be registered via [`SessionState`] -/// to create a custom logical plan for caching. +/// to create a custom logical plan for [`crate::dataframe::DataFrame::cache`]. /// Additionally, a custom [`crate::physical_planner::ExtensionPlanner`]/[`QueryPlanner`] /// may need to be implemented to handle such plans. pub trait CacheFactory: Debug + Send + Sync {