From 3f3a0cfd30d145121ec8f3f9de725ecbf5a335bb Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Thu, 24 Oct 2024 10:35:18 +0800 Subject: [PATCH 01/27] feat: support arbitrary expressions in `LIMIT` plan (#13028) * feat: support arbitrary expressions in `LIMIT` clause * restore test * Fix doc * Update datafusion/optimizer/src/eliminate_limit.rs Co-authored-by: Jax Liu * Update datafusion/expr/src/expr_rewriter/mod.rs Co-authored-by: Jax Liu * Fix clippy * Disallow non-integer types --------- Co-authored-by: Jax Liu --- datafusion/core/src/physical_planner.rs | 25 +++-- .../tests/user_defined/user_defined_plan.rs | 44 ++++---- datafusion/expr/src/expr_rewriter/mod.rs | 9 +- datafusion/expr/src/logical_plan/builder.rs | 19 +++- datafusion/expr/src/logical_plan/display.rs | 6 +- datafusion/expr/src/logical_plan/mod.rs | 4 +- datafusion/expr/src/logical_plan/plan.rs | 105 +++++++++++++++--- datafusion/expr/src/logical_plan/tree_node.rs | 28 ++++- .../optimizer/src/analyzer/type_coercion.rs | 37 +++++- datafusion/optimizer/src/decorrelate.rs | 15 +-- datafusion/optimizer/src/eliminate_limit.rs | 20 ++-- datafusion/optimizer/src/push_down_limit.rs | 54 ++++----- datafusion/proto/src/logical_plan/mod.rs | 23 +++- datafusion/sql/src/query.rs | 92 +++------------ datafusion/sql/src/unparser/plan.rs | 15 +-- datafusion/sql/tests/cases/plan_to_sql.rs | 2 +- datafusion/sqllogictest/test_files/select.slt | 67 ++++++++++- .../substrait/src/logical_plan/consumer.rs | 4 +- .../substrait/src/logical_plan/producer.rs | 15 ++- 19 files changed, 376 insertions(+), 208 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 918ebccbeb70..4a5c156e28ac 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -29,13 +29,12 @@ use crate::error::{DataFusionError, Result}; use crate::execution::context::{ExecutionProps, SessionState}; use crate::logical_expr::utils::generate_sort_key; use crate::logical_expr::{ - Aggregate, EmptyRelation, Join, Projection, Sort, TableScan, Unnest, Window, + Aggregate, EmptyRelation, Join, Projection, Sort, TableScan, Unnest, Values, Window, }; use crate::logical_expr::{ Expr, LogicalPlan, Partitioning as LogicalPartitioning, PlanType, Repartition, UserDefinedLogicalNode, }; -use crate::logical_expr::{Limit, Values}; use crate::physical_expr::{create_physical_expr, create_physical_exprs}; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use crate::physical_plan::analyze::AnalyzeExec; @@ -78,8 +77,8 @@ use datafusion_expr::expr::{ use datafusion_expr::expr_rewriter::unnormalize_cols; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ - DescribeTable, DmlStatement, Extension, Filter, JoinType, RecursiveQuery, SortExpr, - StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, + DescribeTable, DmlStatement, Extension, FetchType, Filter, JoinType, RecursiveQuery, + SkipType, SortExpr, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Literal; @@ -796,8 +795,20 @@ impl DefaultPhysicalPlanner { } LogicalPlan::Subquery(_) => todo!(), LogicalPlan::SubqueryAlias(_) => children.one()?, - LogicalPlan::Limit(Limit { skip, fetch, .. }) => { + LogicalPlan::Limit(limit) => { let input = children.one()?; + let SkipType::Literal(skip) = limit.get_skip_type()? else { + return not_impl_err!( + "Unsupported OFFSET expression: {:?}", + limit.skip + ); + }; + let FetchType::Literal(fetch) = limit.get_fetch_type()? else { + return not_impl_err!( + "Unsupported LIMIT expression: {:?}", + limit.fetch + ); + }; // GlobalLimitExec requires a single partition for input let input = if input.output_partitioning().partition_count() == 1 { @@ -806,13 +817,13 @@ impl DefaultPhysicalPlanner { // Apply a LocalLimitExec to each partition. The optimizer will also insert // a CoalescePartitionsExec between the GlobalLimitExec and LocalLimitExec if let Some(fetch) = fetch { - Arc::new(LocalLimitExec::new(input, *fetch + skip)) + Arc::new(LocalLimitExec::new(input, fetch + skip)) } else { input } }; - Arc::new(GlobalLimitExec::new(input, *skip, *fetch)) + Arc::new(GlobalLimitExec::new(input, skip, fetch)) } LogicalPlan::Unnest(Unnest { list_type_columns, diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 2b45d0ed600b..6c4e3c66e397 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -81,7 +81,7 @@ use datafusion::{ runtime_env::RuntimeEnv, }, logical_expr::{ - Expr, Extension, Limit, LogicalPlan, Sort, UserDefinedLogicalNode, + Expr, Extension, LogicalPlan, Sort, UserDefinedLogicalNode, UserDefinedLogicalNodeCore, }, optimizer::{OptimizerConfig, OptimizerRule}, @@ -98,7 +98,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; use datafusion_expr::tree_node::replace_sort_expression; -use datafusion_expr::{Projection, SortExpr}; +use datafusion_expr::{FetchType, Projection, SortExpr}; use datafusion_optimizer::optimizer::ApplyOrder; use datafusion_optimizer::AnalyzerRule; @@ -361,28 +361,28 @@ impl OptimizerRule for TopKOptimizerRule { // Note: this code simply looks for the pattern of a Limit followed by a // Sort and replaces it by a TopK node. It does not handle many // edge cases (e.g multiple sort columns, sort ASC / DESC), etc. - if let LogicalPlan::Limit(Limit { - fetch: Some(fetch), - input, + let LogicalPlan::Limit(ref limit) = plan else { + return Ok(Transformed::no(plan)); + }; + let FetchType::Literal(Some(fetch)) = limit.get_fetch_type()? else { + return Ok(Transformed::no(plan)); + }; + + if let LogicalPlan::Sort(Sort { + ref expr, + ref input, .. - }) = &plan + }) = limit.input.as_ref() { - if let LogicalPlan::Sort(Sort { - ref expr, - ref input, - .. - }) = **input - { - if expr.len() == 1 { - // we found a sort with a single sort expr, replace with a a TopK - return Ok(Transformed::yes(LogicalPlan::Extension(Extension { - node: Arc::new(TopKPlanNode { - k: *fetch, - input: input.as_ref().clone(), - expr: expr[0].clone(), - }), - }))); - } + if expr.len() == 1 { + // we found a sort with a single sort expr, replace with a a TopK + return Ok(Transformed::yes(LogicalPlan::Extension(Extension { + node: Arc::new(TopKPlanNode { + k: fetch, + input: input.as_ref().clone(), + expr: expr[0].clone(), + }), + }))); } } diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 47cc947be3ca..d6d5c3e2931c 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -306,11 +306,14 @@ impl NamePreserver { /// Create a new NamePreserver for rewriting the `expr` that is part of the specified plan pub fn new(plan: &LogicalPlan) -> Self { Self { - // The schema of Filter, Join and TableScan nodes comes from their inputs rather than - // their expressions, so there is no need to use aliases to preserve expression names. + // The expressions of these plans do not contribute to their output schema, + // so there is no need to preserve expression names to prevent a schema change. use_alias: !matches!( plan, - LogicalPlan::Filter(_) | LogicalPlan::Join(_) | LogicalPlan::TableScan(_) + LogicalPlan::Filter(_) + | LogicalPlan::Join(_) + | LogicalPlan::TableScan(_) + | LogicalPlan::Limit(_) ), } } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index d2ecd56cdc23..cef05b6f8814 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -40,7 +40,7 @@ use crate::utils::{ find_valid_equijoin_key_pair, group_window_expr_by_sort_keys, }; use crate::{ - and, binary_expr, DmlStatement, Expr, ExprSchemable, Operator, RecursiveQuery, + and, binary_expr, lit, DmlStatement, Expr, ExprSchemable, Operator, RecursiveQuery, TableProviderFilterPushDown, TableSource, WriteOp, }; @@ -512,9 +512,22 @@ impl LogicalPlanBuilder { /// `fetch` - Maximum number of rows to fetch, after skipping `skip` rows, /// if specified. pub fn limit(self, skip: usize, fetch: Option) -> Result { + let skip_expr = if skip == 0 { + None + } else { + Some(lit(skip as i64)) + }; + let fetch_expr = fetch.map(|f| lit(f as i64)); + self.limit_by_expr(skip_expr, fetch_expr) + } + + /// Limit the number of rows returned + /// + /// Similar to `limit` but uses expressions for `skip` and `fetch` + pub fn limit_by_expr(self, skip: Option, fetch: Option) -> Result { Ok(Self::new(LogicalPlan::Limit(Limit { - skip, - fetch, + skip: skip.map(Box::new), + fetch: fetch.map(Box::new), input: self.plan, }))) } diff --git a/datafusion/expr/src/logical_plan/display.rs b/datafusion/expr/src/logical_plan/display.rs index 26d54803d403..0287846862af 100644 --- a/datafusion/expr/src/logical_plan/display.rs +++ b/datafusion/expr/src/logical_plan/display.rs @@ -549,11 +549,13 @@ impl<'a, 'b> PgJsonVisitor<'a, 'b> { let mut object = serde_json::json!( { "Node Type": "Limit", - "Skip": skip, } ); + if let Some(s) = skip { + object["Skip"] = s.to_string().into() + }; if let Some(f) = fetch { - object["Fetch"] = serde_json::Value::Number((*f).into()); + object["Fetch"] = f.to_string().into() }; object } diff --git a/datafusion/expr/src/logical_plan/mod.rs b/datafusion/expr/src/logical_plan/mod.rs index da44cfb010d7..18ac3f2ab9cb 100644 --- a/datafusion/expr/src/logical_plan/mod.rs +++ b/datafusion/expr/src/logical_plan/mod.rs @@ -36,9 +36,9 @@ pub use ddl::{ pub use dml::{DmlStatement, WriteOp}; pub use plan::{ projection_schema, Aggregate, Analyze, ColumnUnnestList, CrossJoin, DescribeTable, - Distinct, DistinctOn, EmptyRelation, Explain, Extension, Filter, Join, + Distinct, DistinctOn, EmptyRelation, Explain, Extension, FetchType, Filter, Join, JoinConstraint, JoinType, Limit, LogicalPlan, Partitioning, PlanType, Prepare, - Projection, RecursiveQuery, Repartition, Sort, StringifiedPlan, Subquery, + Projection, RecursiveQuery, Repartition, SkipType, Sort, StringifiedPlan, Subquery, SubqueryAlias, TableScan, ToStringifiedPlan, Union, Unnest, Values, Window, }; pub use statement::{ diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index d8dfe7b56e40..e0aae4cb7448 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -49,7 +49,8 @@ use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion}; use datafusion_common::{ aggregate_functional_dependencies, internal_err, plan_err, Column, Constraints, DFSchema, DFSchemaRef, DataFusionError, Dependency, FunctionalDependence, - FunctionalDependencies, ParamValues, Result, TableReference, UnnestOptions, + FunctionalDependencies, ParamValues, Result, ScalarValue, TableReference, + UnnestOptions, }; use indexmap::IndexSet; @@ -960,11 +961,21 @@ impl LogicalPlan { .map(LogicalPlan::SubqueryAlias) } LogicalPlan::Limit(Limit { skip, fetch, .. }) => { - self.assert_no_expressions(expr)?; + let old_expr_len = skip.iter().chain(fetch.iter()).count(); + if old_expr_len != expr.len() { + return internal_err!( + "Invalid number of new Limit expressions: expected {}, got {}", + old_expr_len, + expr.len() + ); + } + // Pop order is same as the order returned by `LogicalPlan::expressions()` + let new_skip = skip.as_ref().and(expr.pop()); + let new_fetch = fetch.as_ref().and(expr.pop()); let input = self.only_input(inputs)?; Ok(LogicalPlan::Limit(Limit { - skip: *skip, - fetch: *fetch, + skip: new_skip.map(Box::new), + fetch: new_fetch.map(Box::new), input: Arc::new(input), })) } @@ -1339,7 +1350,10 @@ impl LogicalPlan { LogicalPlan::RecursiveQuery(_) => None, LogicalPlan::Subquery(_) => None, LogicalPlan::SubqueryAlias(SubqueryAlias { input, .. }) => input.max_rows(), - LogicalPlan::Limit(Limit { fetch, .. }) => *fetch, + LogicalPlan::Limit(limit) => match limit.get_fetch_type() { + Ok(FetchType::Literal(s)) => s, + _ => None, + }, LogicalPlan::Distinct( Distinct::All(input) | Distinct::On(DistinctOn { input, .. }), ) => input.max_rows(), @@ -1909,16 +1923,20 @@ impl LogicalPlan { ) } }, - LogicalPlan::Limit(Limit { - ref skip, - ref fetch, - .. - }) => { + LogicalPlan::Limit(limit) => { + // Attempt to display `skip` and `fetch` as literals if possible, otherwise as expressions. + let skip_str = match limit.get_skip_type() { + Ok(SkipType::Literal(n)) => n.to_string(), + _ => limit.skip.as_ref().map_or_else(|| "None".to_string(), |x| x.to_string()), + }; + let fetch_str = match limit.get_fetch_type() { + Ok(FetchType::Literal(Some(n))) => n.to_string(), + Ok(FetchType::Literal(None)) => "None".to_string(), + _ => limit.fetch.as_ref().map_or_else(|| "None".to_string(), |x| x.to_string()) + }; write!( f, - "Limit: skip={}, fetch={}", - skip, - fetch.map_or_else(|| "None".to_string(), |x| x.to_string()) + "Limit: skip={}, fetch={}", skip_str,fetch_str, ) } LogicalPlan::Subquery(Subquery { .. }) => { @@ -2778,14 +2796,71 @@ impl PartialOrd for Extension { #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct Limit { /// Number of rows to skip before fetch - pub skip: usize, + pub skip: Option>, /// Maximum number of rows to fetch, /// None means fetching all rows - pub fetch: Option, + pub fetch: Option>, /// The logical plan pub input: Arc, } +/// Different types of skip expression in Limit plan. +pub enum SkipType { + /// The skip expression is a literal value. + Literal(usize), + /// Currently only supports expressions that can be folded into constants. + UnsupportedExpr, +} + +/// Different types of fetch expression in Limit plan. +pub enum FetchType { + /// The fetch expression is a literal value. + /// `Literal(None)` means the fetch expression is not provided. + Literal(Option), + /// Currently only supports expressions that can be folded into constants. + UnsupportedExpr, +} + +impl Limit { + /// Get the skip type from the limit plan. + pub fn get_skip_type(&self) -> Result { + match self.skip.as_deref() { + Some(expr) => match *expr { + Expr::Literal(ScalarValue::Int64(s)) => { + // `skip = NULL` is equivalent to `skip = 0` + let s = s.unwrap_or(0); + if s >= 0 { + Ok(SkipType::Literal(s as usize)) + } else { + plan_err!("OFFSET must be >=0, '{}' was provided", s) + } + } + _ => Ok(SkipType::UnsupportedExpr), + }, + // `skip = None` is equivalent to `skip = 0` + None => Ok(SkipType::Literal(0)), + } + } + + /// Get the fetch type from the limit plan. + pub fn get_fetch_type(&self) -> Result { + match self.fetch.as_deref() { + Some(expr) => match *expr { + Expr::Literal(ScalarValue::Int64(Some(s))) => { + if s >= 0 { + Ok(FetchType::Literal(Some(s as usize))) + } else { + plan_err!("LIMIT must be >= 0, '{}' was provided", s) + } + } + Expr::Literal(ScalarValue::Int64(None)) => Ok(FetchType::Literal(None)), + _ => Ok(FetchType::UnsupportedExpr), + }, + None => Ok(FetchType::Literal(None)), + } + } +} + /// Removes duplicate rows from the input #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum Distinct { diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 606868e75abf..b8d7043d7746 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -43,6 +43,7 @@ use crate::{ Repartition, Sort, Subquery, SubqueryAlias, TableScan, Union, Unnest, UserDefinedLogicalNode, Values, Window, }; +use std::ops::Deref; use std::sync::Arc; use crate::expr::{Exists, InSubquery}; @@ -515,12 +516,16 @@ impl LogicalPlan { .chain(select_expr.iter()) .chain(sort_expr.iter().flatten().map(|sort| &sort.expr)) .apply_until_stop(f), + LogicalPlan::Limit(Limit { skip, fetch, .. }) => skip + .iter() + .chain(fetch.iter()) + .map(|e| e.deref()) + .apply_until_stop(f), // plans without expressions LogicalPlan::EmptyRelation(_) | LogicalPlan::RecursiveQuery(_) | LogicalPlan::Subquery(_) | LogicalPlan::SubqueryAlias(_) - | LogicalPlan::Limit(_) | LogicalPlan::Statement(_) | LogicalPlan::CrossJoin(_) | LogicalPlan::Analyze(_) @@ -726,13 +731,32 @@ impl LogicalPlan { schema, })) }), + LogicalPlan::Limit(Limit { skip, fetch, input }) => { + let skip = skip.map(|e| *e); + let fetch = fetch.map(|e| *e); + map_until_stop_and_collect!( + skip.map_or(Ok::<_, DataFusionError>(Transformed::no(None)), |e| { + Ok(f(e)?.update_data(Some)) + }), + fetch, + fetch.map_or(Ok::<_, DataFusionError>(Transformed::no(None)), |e| { + Ok(f(e)?.update_data(Some)) + }) + )? + .update_data(|(skip, fetch)| { + LogicalPlan::Limit(Limit { + skip: skip.map(Box::new), + fetch: fetch.map(Box::new), + input, + }) + }) + } // plans without expressions LogicalPlan::EmptyRelation(_) | LogicalPlan::Unnest(_) | LogicalPlan::RecursiveQuery(_) | LogicalPlan::Subquery(_) | LogicalPlan::SubqueryAlias(_) - | LogicalPlan::Limit(_) | LogicalPlan::Statement(_) | LogicalPlan::CrossJoin(_) | LogicalPlan::Analyze(_) diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index e5d280289342..36b72233b5af 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -51,8 +51,9 @@ use datafusion_expr::type_coercion::{is_datetime, is_utf8_or_large_utf8}; use datafusion_expr::utils::merge_schema; use datafusion_expr::{ is_false, is_not_false, is_not_true, is_not_unknown, is_true, is_unknown, not, - AggregateUDF, Expr, ExprFunctionExt, ExprSchemable, Join, LogicalPlan, Operator, - Projection, ScalarUDF, Union, WindowFrame, WindowFrameBound, WindowFrameUnits, + AggregateUDF, Expr, ExprFunctionExt, ExprSchemable, Join, Limit, LogicalPlan, + Operator, Projection, ScalarUDF, Union, WindowFrame, WindowFrameBound, + WindowFrameUnits, }; /// Performs type coercion by determining the schema @@ -169,6 +170,7 @@ impl<'a> TypeCoercionRewriter<'a> { match plan { LogicalPlan::Join(join) => self.coerce_join(join), LogicalPlan::Union(union) => Self::coerce_union(union), + LogicalPlan::Limit(limit) => Self::coerce_limit(limit), _ => Ok(plan), } } @@ -230,6 +232,37 @@ impl<'a> TypeCoercionRewriter<'a> { })) } + /// Coerce the fetch and skip expression to Int64 type. + fn coerce_limit(limit: Limit) -> Result { + fn coerce_limit_expr( + expr: Expr, + schema: &DFSchema, + expr_name: &str, + ) -> Result { + let dt = expr.get_type(schema)?; + if dt.is_integer() || dt.is_null() { + expr.cast_to(&DataType::Int64, schema) + } else { + plan_err!("Expected {expr_name} to be an integer or null, but got {dt:?}") + } + } + + let empty_schema = DFSchema::empty(); + let new_fetch = limit + .fetch + .map(|expr| coerce_limit_expr(*expr, &empty_schema, "LIMIT")) + .transpose()?; + let new_skip = limit + .skip + .map(|expr| coerce_limit_expr(*expr, &empty_schema, "OFFSET")) + .transpose()?; + Ok(LogicalPlan::Limit(Limit { + input: limit.input, + fetch: new_fetch.map(Box::new), + skip: new_skip.map(Box::new), + })) + } + fn coerce_join_filter(&self, expr: Expr) -> Result { let expr_type = expr.get_type(self.schema)?; match expr_type { diff --git a/datafusion/optimizer/src/decorrelate.rs b/datafusion/optimizer/src/decorrelate.rs index 7f918c03e3ac..baf449a045eb 100644 --- a/datafusion/optimizer/src/decorrelate.rs +++ b/datafusion/optimizer/src/decorrelate.rs @@ -31,7 +31,9 @@ use datafusion_common::{plan_err, Column, DFSchemaRef, Result, ScalarValue}; use datafusion_expr::expr::Alias; use datafusion_expr::simplify::SimplifyContext; use datafusion_expr::utils::{conjunction, find_join_exprs, split_conjunction}; -use datafusion_expr::{expr, lit, EmptyRelation, Expr, LogicalPlan, LogicalPlanBuilder}; +use datafusion_expr::{ + expr, lit, EmptyRelation, Expr, FetchType, LogicalPlan, LogicalPlanBuilder, +}; use datafusion_physical_expr::execution_props::ExecutionProps; /// This struct rewrite the sub query plan by pull up the correlated @@ -327,16 +329,15 @@ impl TreeNodeRewriter for PullUpCorrelatedExpr { let new_plan = match (self.exists_sub_query, self.join_filters.is_empty()) { // Correlated exist subquery, remove the limit(so that correlated expressions can pull up) - (true, false) => Transformed::yes( - if limit.fetch.filter(|limit_row| *limit_row == 0).is_some() { + (true, false) => Transformed::yes(match limit.get_fetch_type()? { + FetchType::Literal(Some(0)) => { LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, schema: Arc::clone(limit.input.schema()), }) - } else { - LogicalPlanBuilder::from((*limit.input).clone()).build()? - }, - ), + } + _ => LogicalPlanBuilder::from((*limit.input).clone()).build()?, + }), _ => Transformed::no(plan), }; if let Some(input_map) = input_expr_map { diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index 25304d4ccafa..829d4c2d2217 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -20,7 +20,7 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::Transformed; use datafusion_common::Result; -use datafusion_expr::logical_plan::{EmptyRelation, LogicalPlan}; +use datafusion_expr::logical_plan::{EmptyRelation, FetchType, LogicalPlan, SkipType}; use std::sync::Arc; /// Optimizer rule to replace `LIMIT 0` or `LIMIT` whose ancestor LIMIT's skip is @@ -63,8 +63,13 @@ impl OptimizerRule for EliminateLimit { > { match plan { LogicalPlan::Limit(limit) => { - if let Some(fetch) = limit.fetch { - if fetch == 0 { + // Only supports rewriting for literal fetch + let FetchType::Literal(fetch) = limit.get_fetch_type()? else { + return Ok(Transformed::no(LogicalPlan::Limit(limit))); + }; + + if let Some(v) = fetch { + if v == 0 { return Ok(Transformed::yes(LogicalPlan::EmptyRelation( EmptyRelation { produce_one_row: false, @@ -72,11 +77,10 @@ impl OptimizerRule for EliminateLimit { }, ))); } - } else if limit.skip == 0 { - // input also can be Limit, so we should apply again. - return Ok(self - .rewrite(Arc::unwrap_or_clone(limit.input), _config) - .unwrap()); + } else if matches!(limit.get_skip_type()?, SkipType::Literal(0)) { + // If fetch is `None` and skip is 0, then Limit takes no effect and + // we can remove it. Its input also can be Limit, so we should apply again. + return self.rewrite(Arc::unwrap_or_clone(limit.input), _config); } Ok(Transformed::no(LogicalPlan::Limit(limit))) } diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index 6ed77387046e..bf5ce0531e06 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -27,6 +27,7 @@ use datafusion_common::tree_node::Transformed; use datafusion_common::utils::combine_limit; use datafusion_common::Result; use datafusion_expr::logical_plan::{Join, JoinType, Limit, LogicalPlan}; +use datafusion_expr::{lit, FetchType, SkipType}; /// Optimization rule that tries to push down `LIMIT`. /// @@ -56,16 +57,27 @@ impl OptimizerRule for PushDownLimit { return Ok(Transformed::no(plan)); }; - let Limit { skip, fetch, input } = limit; + // Currently only rewrite if skip and fetch are both literals + let SkipType::Literal(skip) = limit.get_skip_type()? else { + return Ok(Transformed::no(LogicalPlan::Limit(limit))); + }; + let FetchType::Literal(fetch) = limit.get_fetch_type()? else { + return Ok(Transformed::no(LogicalPlan::Limit(limit))); + }; // Merge the Parent Limit and the Child Limit. - if let LogicalPlan::Limit(child) = input.as_ref() { - let (skip, fetch) = - combine_limit(limit.skip, limit.fetch, child.skip, child.fetch); - + if let LogicalPlan::Limit(child) = limit.input.as_ref() { + let SkipType::Literal(child_skip) = child.get_skip_type()? else { + return Ok(Transformed::no(LogicalPlan::Limit(limit))); + }; + let FetchType::Literal(child_fetch) = child.get_fetch_type()? else { + return Ok(Transformed::no(LogicalPlan::Limit(limit))); + }; + + let (skip, fetch) = combine_limit(skip, fetch, child_skip, child_fetch); let plan = LogicalPlan::Limit(Limit { - skip, - fetch, + skip: Some(Box::new(lit(skip as i64))), + fetch: fetch.map(|f| Box::new(lit(f as i64))), input: Arc::clone(&child.input), }); @@ -75,14 +87,10 @@ impl OptimizerRule for PushDownLimit { // no fetch to push, so return the original plan let Some(fetch) = fetch else { - return Ok(Transformed::no(LogicalPlan::Limit(Limit { - skip, - fetch, - input, - }))); + return Ok(Transformed::no(LogicalPlan::Limit(limit))); }; - match Arc::unwrap_or_clone(input) { + match Arc::unwrap_or_clone(limit.input) { LogicalPlan::TableScan(mut scan) => { let rows_needed = if fetch != 0 { fetch + skip } else { 0 }; let new_fetch = scan @@ -162,8 +170,8 @@ impl OptimizerRule for PushDownLimit { .into_iter() .map(|child| { LogicalPlan::Limit(Limit { - skip: 0, - fetch: Some(fetch + skip), + skip: None, + fetch: Some(Box::new(lit((fetch + skip) as i64))), input: Arc::new(child.clone()), }) }) @@ -203,8 +211,8 @@ impl OptimizerRule for PushDownLimit { /// ``` fn make_limit(skip: usize, fetch: usize, input: Arc) -> LogicalPlan { LogicalPlan::Limit(Limit { - skip, - fetch: Some(fetch), + skip: Some(Box::new(lit(skip as i64))), + fetch: Some(Box::new(lit(fetch as i64))), input, }) } @@ -224,11 +232,7 @@ fn original_limit( fetch: usize, input: LogicalPlan, ) -> Result> { - Ok(Transformed::no(LogicalPlan::Limit(Limit { - skip, - fetch: Some(fetch), - input: Arc::new(input), - }))) + Ok(Transformed::no(make_limit(skip, fetch, Arc::new(input)))) } /// Returns the a transformed limit @@ -237,11 +241,7 @@ fn transformed_limit( fetch: usize, input: LogicalPlan, ) -> Result> { - Ok(Transformed::yes(LogicalPlan::Limit(Limit { - skip, - fetch: Some(fetch), - input: Arc::new(input), - }))) + Ok(Transformed::yes(make_limit(skip, fetch, Arc::new(input)))) } /// Adds a limit to the inputs of a join, if possible diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 4adbb9318d51..73df506397b1 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -62,13 +62,13 @@ use datafusion_expr::{ logical_plan::{ builder::project, Aggregate, CreateCatalog, CreateCatalogSchema, CreateExternalTable, CreateView, CrossJoin, DdlStatement, Distinct, - EmptyRelation, Extension, Join, JoinConstraint, Limit, Prepare, Projection, - Repartition, Sort, SubqueryAlias, TableScan, Values, Window, + EmptyRelation, Extension, Join, JoinConstraint, Prepare, Projection, Repartition, + Sort, SubqueryAlias, TableScan, Values, Window, }, DistinctOn, DropView, Expr, LogicalPlan, LogicalPlanBuilder, ScalarUDF, SortExpr, WindowUDF, }; -use datafusion_expr::{AggregateUDF, ColumnUnnestList, Unnest}; +use datafusion_expr::{AggregateUDF, ColumnUnnestList, FetchType, SkipType, Unnest}; use self::to_proto::{serialize_expr, serialize_exprs}; use crate::logical_plan::to_proto::serialize_sorts; @@ -1265,17 +1265,28 @@ impl AsLogicalPlan for LogicalPlanNode { ))), }) } - LogicalPlan::Limit(Limit { input, skip, fetch }) => { + LogicalPlan::Limit(limit) => { let input: protobuf::LogicalPlanNode = protobuf::LogicalPlanNode::try_from_logical_plan( - input.as_ref(), + limit.input.as_ref(), extension_codec, )?; + let SkipType::Literal(skip) = limit.get_skip_type()? else { + return Err(proto_error( + "LogicalPlan::Limit only supports literal skip values", + )); + }; + let FetchType::Literal(fetch) = limit.get_fetch_type()? else { + return Err(proto_error( + "LogicalPlan::Limit only supports literal fetch values", + )); + }; + Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Limit(Box::new( protobuf::LimitNode { input: Some(Box::new(input)), - skip: *skip as i64, + skip: skip as i64, fetch: fetch.unwrap_or(i64::MAX as usize) as i64, }, ))), diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index 54945ec43d10..842a1c0cbec1 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -19,15 +19,14 @@ use std::sync::Arc; use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use datafusion_common::{not_impl_err, plan_err, Constraints, Result, ScalarValue}; +use datafusion_common::{not_impl_err, Constraints, DFSchema, Result}; use datafusion_expr::expr::Sort; use datafusion_expr::{ - CreateMemoryTable, DdlStatement, Distinct, Expr, LogicalPlan, LogicalPlanBuilder, - Operator, + CreateMemoryTable, DdlStatement, Distinct, LogicalPlan, LogicalPlanBuilder, }; use sqlparser::ast::{ Expr as SQLExpr, Offset as SQLOffset, OrderBy, OrderByExpr, Query, SelectInto, - SetExpr, Value, + SetExpr, }; impl<'a, S: ContextProvider> SqlToRel<'a, S> { @@ -85,35 +84,18 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { return Ok(input); } - let skip = match skip { - Some(skip_expr) => { - let expr = self.sql_to_expr( - skip_expr.value, - input.schema(), - &mut PlannerContext::new(), - )?; - let n = get_constant_result(&expr, "OFFSET")?; - convert_usize_with_check(n, "OFFSET") - } - _ => Ok(0), - }?; - - let fetch = match fetch { - Some(limit_expr) - if limit_expr != sqlparser::ast::Expr::Value(Value::Null) => - { - let expr = self.sql_to_expr( - limit_expr, - input.schema(), - &mut PlannerContext::new(), - )?; - let n = get_constant_result(&expr, "LIMIT")?; - Some(convert_usize_with_check(n, "LIMIT")?) - } - _ => None, - }; - - LogicalPlanBuilder::from(input).limit(skip, fetch)?.build() + // skip and fetch expressions are not allowed to reference columns from the input plan + let empty_schema = DFSchema::empty(); + + let skip = skip + .map(|o| self.sql_to_expr(o.value, &empty_schema, &mut PlannerContext::new())) + .transpose()?; + let fetch = fetch + .map(|e| self.sql_to_expr(e, &empty_schema, &mut PlannerContext::new())) + .transpose()?; + LogicalPlanBuilder::from(input) + .limit_by_expr(skip, fetch)? + .build() } /// Wrap the logical in a sort @@ -159,50 +141,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } -/// Retrieves the constant result of an expression, evaluating it if possible. -/// -/// This function takes an expression and an argument name as input and returns -/// a `Result` indicating either the constant result of the expression or an -/// error if the expression cannot be evaluated. -/// -/// # Arguments -/// -/// * `expr` - An `Expr` representing the expression to evaluate. -/// * `arg_name` - The name of the argument for error messages. -/// -/// # Returns -/// -/// * `Result` - An `Ok` variant containing the constant result if evaluation is successful, -/// or an `Err` variant containing an error message if evaluation fails. -/// -/// tracks a more general solution -fn get_constant_result(expr: &Expr, arg_name: &str) -> Result { - match expr { - Expr::Literal(ScalarValue::Int64(Some(s))) => Ok(*s), - Expr::BinaryExpr(binary_expr) => { - let lhs = get_constant_result(&binary_expr.left, arg_name)?; - let rhs = get_constant_result(&binary_expr.right, arg_name)?; - let res = match binary_expr.op { - Operator::Plus => lhs + rhs, - Operator::Minus => lhs - rhs, - Operator::Multiply => lhs * rhs, - _ => return plan_err!("Unsupported operator for {arg_name} clause"), - }; - Ok(res) - } - _ => plan_err!("Unexpected expression in {arg_name} clause"), - } -} - -/// Converts an `i64` to `usize`, performing a boundary check. -fn convert_usize_with_check(n: i64, arg_name: &str) -> Result { - if n < 0 { - plan_err!("{arg_name} must be >= 0, '{n}' was provided.") - } else { - Ok(n as usize) - } -} - /// Returns the order by expressions from the query. fn to_order_by_exprs(order_by: Option) -> Result> { let Some(OrderBy { exprs, interpolate }) = order_by else { diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index 037748035fbf..0147a607567b 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -343,20 +343,16 @@ impl Unparser<'_> { relation, ); } - - if let Some(fetch) = limit.fetch { + if let Some(fetch) = &limit.fetch { let Some(query) = query.as_mut() else { return internal_err!( "Limit operator only valid in a statement context." ); }; - query.limit(Some(ast::Expr::Value(ast::Value::Number( - fetch.to_string(), - false, - )))); + query.limit(Some(self.expr_to_sql(fetch)?)); } - if limit.skip > 0 { + if let Some(skip) = &limit.skip { let Some(query) = query.as_mut() else { return internal_err!( "Offset operator only valid in a statement context." @@ -364,10 +360,7 @@ impl Unparser<'_> { }; query.offset(Some(ast::Offset { rows: ast::OffsetRows::None, - value: ast::Expr::Value(ast::Value::Number( - limit.skip.to_string(), - false, - )), + value: self.expr_to_sql(skip)?, })); } diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index e7b96199511a..9ed084eec249 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -1016,7 +1016,7 @@ fn test_without_offset() { #[test] fn test_with_offset0() { - sql_round_trip(MySqlDialect {}, "select 1 offset 0", "SELECT 1"); + sql_round_trip(MySqlDialect {}, "select 1 offset 0", "SELECT 1 OFFSET 0"); } #[test] diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 9910ca8da71f..f2ab4135aaa7 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -581,9 +581,32 @@ select * from (select 1 a union all select 2) b order by a limit 1; 1 # select limit clause invalid -statement error DataFusion error: Error during planning: LIMIT must be >= 0, '\-1' was provided\. +statement error Error during planning: LIMIT must be >= 0, '-1' was provided select * from (select 1 a union all select 2) b order by a limit -1; +statement error Error during planning: OFFSET must be >=0, '-1' was provided +select * from (select 1 a union all select 2) b order by a offset -1; + +statement error Unsupported LIMIT expression +select * from (values(1),(2)) limit (select 1); + +statement error Unsupported OFFSET expression +select * from (values(1),(2)) offset (select 1); + +# disallow non-integer limit/offset +statement error Expected LIMIT to be an integer or null, but got Float64 +select * from (values(1),(2)) limit 0.5; + +statement error Expected OFFSET to be an integer or null, but got Utf8 +select * from (values(1),(2)) offset '1'; + +# test with different integer types +query I +select * from (values (1), (2), (3), (4)) limit 2::int OFFSET 1::tinyint +---- +2 +3 + # select limit with basic arithmetic query I select * from (select 1 a union all select 2) b order by a limit 1+1; @@ -597,13 +620,38 @@ select * from (values (1)) LIMIT 10*100; ---- 1 -# More complex expressions in the limit is not supported yet. -# See issue: https://github.com/apache/datafusion/issues/9821 -statement error DataFusion error: Error during planning: Unsupported operator for LIMIT clause +# select limit with complex arithmetic +query I select * from (values (1)) LIMIT 100/10; +---- +1 -# More complex expressions in the limit is not supported yet. -statement error DataFusion error: Error during planning: Unexpected expression in LIMIT clause +# test constant-folding of LIMIT expr +query I +select * from (values (1), (2), (3), (4)) LIMIT abs(-4) + 4 / -2; -- LIMIT 2 +---- +1 +2 + +# test constant-folding of OFFSET expr +query I +select * from (values (1), (2), (3), (4)) OFFSET abs(-4) + 4 / -2; -- OFFSET 2 +---- +3 +4 + +# test constant-folding of LIMIT and OFFSET +query I +select * from (values (1), (2), (3), (4)) + -- LIMIT 2 + LIMIT abs(-4) + -1 * 2 + -- OFFSET 1 + OFFSET case when 1 < 2 then 1 else 0 end; +---- +2 +3 + +statement error Schema error: No field named column1. select * from (values (1)) LIMIT cast(column1 as tinyint); # select limit clause @@ -613,6 +661,13 @@ select * from (select 1 a union all select 2) b order by a limit null; 1 2 +# offset null takes no effect +query I +select * from (select 1 a union all select 2) b order by a offset null; +---- +1 +2 + # select limit clause query I select * from (select 1 a union all select 2) b order by a limit 0; diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 8a8d195507a2..3d5d7cce5673 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -623,8 +623,8 @@ pub async fn from_substrait_rel( from_substrait_rel(ctx, input, extensions).await?, ); let offset = fetch.offset as usize; - // Since protobuf can't directly distinguish `None` vs `0` `None` is encoded as `MAX` - let count = if fetch.count as usize == usize::MAX { + // -1 means that ALL records should be returned + let count = if fetch.count == -1 { None } else { Some(fetch.count as usize) diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 7504a287c055..bb50c4b9610f 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -24,7 +24,7 @@ use substrait::proto::expression_reference::ExprType; use arrow_buffer::ToByteSlice; use datafusion::arrow::datatypes::{Field, IntervalUnit}; use datafusion::logical_expr::{ - CrossJoin, Distinct, Like, Partitioning, WindowFrameUnits, + CrossJoin, Distinct, FetchType, Like, Partitioning, SkipType, WindowFrameUnits, }; use datafusion::{ arrow::datatypes::{DataType, TimeUnit}, @@ -326,14 +326,19 @@ pub fn to_substrait_rel( } LogicalPlan::Limit(limit) => { let input = to_substrait_rel(limit.input.as_ref(), ctx, extensions)?; - // Since protobuf can't directly distinguish `None` vs `0` encode `None` as `MAX` - let limit_fetch = limit.fetch.unwrap_or(usize::MAX); + let FetchType::Literal(fetch) = limit.get_fetch_type()? else { + return not_impl_err!("Non-literal limit fetch"); + }; + let SkipType::Literal(skip) = limit.get_skip_type()? else { + return not_impl_err!("Non-literal limit skip"); + }; Ok(Box::new(Rel { rel_type: Some(RelType::Fetch(Box::new(FetchRel { common: None, input: Some(input), - offset: limit.skip as i64, - count: limit_fetch as i64, + offset: skip as i64, + // use -1 to signal that ALL records should be returned + count: fetch.map(|f| f as i64).unwrap_or(-1), advanced_extension: None, }))), })) From 8adbc2324afac66d8cb88b20cb1482913b190d4b Mon Sep 17 00:00:00 2001 From: Mustafa Akur <33904309+akurmustafa@users.noreply.github.com> Date: Thu, 24 Oct 2024 00:10:09 -0700 Subject: [PATCH 02/27] [minor]: use arrow take_batch instead of get_record_batch_indices (#13084) * Initial commit * Fix linter errors * Minor changes * Fix error --- datafusion/common/src/utils/mod.rs | 24 ++++--------------- .../tests/fuzz_cases/equivalence/utils.rs | 17 ++++--------- .../src/windows/bounded_window_agg_exec.rs | 6 ++--- 3 files changed, 12 insertions(+), 35 deletions(-) diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index def1def9853c..dacf90af9bbf 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -23,16 +23,14 @@ pub mod proxy; pub mod string_utils; use crate::error::{_internal_datafusion_err, _internal_err}; -use crate::{arrow_datafusion_err, DataFusionError, Result, ScalarValue}; -use arrow::array::{ArrayRef, PrimitiveArray}; +use crate::{DataFusionError, Result, ScalarValue}; +use arrow::array::ArrayRef; use arrow::buffer::OffsetBuffer; -use arrow::compute::{partition, take_arrays, SortColumn, SortOptions}; -use arrow::datatypes::{Field, SchemaRef, UInt32Type}; -use arrow::record_batch::RecordBatch; +use arrow::compute::{partition, SortColumn, SortOptions}; +use arrow::datatypes::{Field, SchemaRef}; use arrow_array::cast::AsArray; use arrow_array::{ Array, FixedSizeListArray, LargeListArray, ListArray, OffsetSizeTrait, - RecordBatchOptions, }; use arrow_schema::DataType; use sqlparser::ast::Ident; @@ -92,20 +90,6 @@ pub fn get_row_at_idx(columns: &[ArrayRef], idx: usize) -> Result, -) -> Result { - let new_columns = take_arrays(record_batch.columns(), indices, None)?; - RecordBatch::try_new_with_options( - record_batch.schema(), - new_columns, - &RecordBatchOptions::new().with_row_count(Some(indices.len())), - ) - .map_err(|e| arrow_datafusion_err!(e)) -} - /// This function compares two tuples depending on the given sort options. pub fn compare_rows( x: &[ScalarValue], diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 61691311fe4e..acc45fe0e591 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -22,15 +22,11 @@ use std::any::Any; use std::cmp::Ordering; use std::sync::Arc; -use arrow::compute::{lexsort_to_indices, SortColumn}; +use arrow::compute::{lexsort_to_indices, take_record_batch, SortColumn}; use arrow::datatypes::{DataType, Field, Schema}; -use arrow_array::{ - ArrayRef, Float32Array, Float64Array, PrimitiveArray, RecordBatch, UInt32Array, -}; +use arrow_array::{ArrayRef, Float32Array, Float64Array, RecordBatch, UInt32Array}; use arrow_schema::{SchemaRef, SortOptions}; -use datafusion_common::utils::{ - compare_rows, get_record_batch_at_indices, get_row_at_idx, -}; +use datafusion_common::utils::{compare_rows, get_row_at_idx}; use datafusion_common::{exec_err, plan_datafusion_err, DataFusionError, Result}; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; @@ -465,7 +461,7 @@ pub fn generate_table_for_orderings( // Sort batch according to first ordering expression let sort_columns = get_sort_columns(&batch, &orderings[0])?; let sort_indices = lexsort_to_indices(&sort_columns, None)?; - let mut batch = get_record_batch_at_indices(&batch, &sort_indices)?; + let mut batch = take_record_batch(&batch, &sort_indices)?; // prune out rows that is invalid according to remaining orderings. for ordering in orderings.iter().skip(1) { @@ -490,10 +486,7 @@ pub fn generate_table_for_orderings( } } // Only keep valid rows, that satisfies given ordering relation. - batch = get_record_batch_at_indices( - &batch, - &PrimitiveArray::from_iter_values(keep_indices), - )?; + batch = take_record_batch(&batch, &UInt32Array::from_iter_values(keep_indices))?; } Ok(batch) diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 6254ae139a00..6495657339fa 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -40,6 +40,7 @@ use crate::{ SendableRecordBatchStream, Statistics, WindowExpr, }; use ahash::RandomState; +use arrow::compute::take_record_batch; use arrow::{ array::{Array, ArrayRef, RecordBatchOptions, UInt32Builder}, compute::{concat, concat_batches, sort_to_indices, take_arrays}, @@ -49,8 +50,7 @@ use arrow::{ use datafusion_common::hash_utils::create_hashes; use datafusion_common::stats::Precision; use datafusion_common::utils::{ - evaluate_partition_ranges, get_at_indices, get_record_batch_at_indices, - get_row_at_idx, + evaluate_partition_ranges, get_at_indices, get_row_at_idx, }; use datafusion_common::{arrow_datafusion_err, exec_err, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -558,7 +558,7 @@ impl PartitionSearcher for LinearSearch { let mut new_indices = UInt32Builder::with_capacity(indices.len()); new_indices.append_slice(&indices); let indices = new_indices.finish(); - Ok((row, get_record_batch_at_indices(record_batch, &indices)?)) + Ok((row, take_record_batch(record_batch, &indices)?)) }) .collect() } From f2da32b3bde851c34e9df0a2f4c174a5392f8897 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Burak=20=C5=9Een?= Date: Thu, 24 Oct 2024 10:57:57 +0300 Subject: [PATCH 03/27] deprecated (#13076) --- datafusion-examples/examples/sql_analysis.rs | 6 +- datafusion/core/src/physical_planner.rs | 4 -- datafusion/expr/src/logical_plan/display.rs | 5 -- datafusion/expr/src/logical_plan/mod.rs | 4 +- datafusion/expr/src/logical_plan/plan.rs | 67 +------------------ datafusion/expr/src/logical_plan/tree_node.rs | 28 ++------ datafusion/optimizer/src/analyzer/subquery.rs | 1 - .../optimizer/src/common_subexpr_eliminate.rs | 1 - .../optimizer/src/eliminate_cross_join.rs | 35 +++------- .../optimizer/src/optimize_projections/mod.rs | 11 --- .../optimizer/src/propagate_empty_relation.rs | 13 ---- datafusion/optimizer/src/push_down_filter.rs | 66 ++---------------- datafusion/optimizer/src/push_down_limit.rs | 7 -- datafusion/proto/src/logical_plan/mod.rs | 24 +------ datafusion/sql/src/unparser/plan.rs | 38 ----------- .../substrait/src/logical_plan/producer.rs | 21 +----- 16 files changed, 32 insertions(+), 299 deletions(-) diff --git a/datafusion-examples/examples/sql_analysis.rs b/datafusion-examples/examples/sql_analysis.rs index 9a2aabaa79c2..2158b8e4b016 100644 --- a/datafusion-examples/examples/sql_analysis.rs +++ b/datafusion-examples/examples/sql_analysis.rs @@ -39,7 +39,7 @@ fn total_join_count(plan: &LogicalPlan) -> usize { // We can use the TreeNode API to walk over a LogicalPlan. plan.apply(|node| { // if we encounter a join we update the running count - if matches!(node, LogicalPlan::Join(_) | LogicalPlan::CrossJoin(_)) { + if matches!(node, LogicalPlan::Join(_)) { total += 1; } Ok(TreeNodeRecursion::Continue) @@ -89,7 +89,7 @@ fn count_trees(plan: &LogicalPlan) -> (usize, Vec) { while let Some(node) = to_visit.pop() { // if we encounter a join, we know were at the root of the tree // count this tree and recurse on it's inputs - if matches!(node, LogicalPlan::Join(_) | LogicalPlan::CrossJoin(_)) { + if matches!(node, LogicalPlan::Join(_)) { let (group_count, inputs) = count_tree(node); total += group_count; groups.push(group_count); @@ -151,7 +151,7 @@ fn count_tree(join: &LogicalPlan) -> (usize, Vec<&LogicalPlan>) { } // any join we count - if matches!(node, LogicalPlan::Join(_) | LogicalPlan::CrossJoin(_)) { + if matches!(node, LogicalPlan::Join(_)) { total += 1; Ok(TreeNodeRecursion::Continue) } else { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 4a5c156e28ac..5a4ae868d04a 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1127,10 +1127,6 @@ impl DefaultPhysicalPlanner { join } } - LogicalPlan::CrossJoin(_) => { - let [left, right] = children.two()?; - Arc::new(CrossJoinExec::new(left, right)) - } LogicalPlan::RecursiveQuery(RecursiveQuery { name, is_distinct, .. }) => { diff --git a/datafusion/expr/src/logical_plan/display.rs b/datafusion/expr/src/logical_plan/display.rs index 0287846862af..c0549451a776 100644 --- a/datafusion/expr/src/logical_plan/display.rs +++ b/datafusion/expr/src/logical_plan/display.rs @@ -504,11 +504,6 @@ impl<'a, 'b> PgJsonVisitor<'a, 'b> { "Filter": format!("{}", filter_expr) }) } - LogicalPlan::CrossJoin(_) => { - json!({ - "Node Type": "Cross Join" - }) - } LogicalPlan::Repartition(Repartition { partitioning_scheme, .. diff --git a/datafusion/expr/src/logical_plan/mod.rs b/datafusion/expr/src/logical_plan/mod.rs index 18ac3f2ab9cb..80a896212442 100644 --- a/datafusion/expr/src/logical_plan/mod.rs +++ b/datafusion/expr/src/logical_plan/mod.rs @@ -35,8 +35,8 @@ pub use ddl::{ }; pub use dml::{DmlStatement, WriteOp}; pub use plan::{ - projection_schema, Aggregate, Analyze, ColumnUnnestList, CrossJoin, DescribeTable, - Distinct, DistinctOn, EmptyRelation, Explain, Extension, FetchType, Filter, Join, + projection_schema, Aggregate, Analyze, ColumnUnnestList, DescribeTable, Distinct, + DistinctOn, EmptyRelation, Explain, Extension, FetchType, Filter, Join, JoinConstraint, JoinType, Limit, LogicalPlan, Partitioning, PlanType, Prepare, Projection, RecursiveQuery, Repartition, SkipType, Sort, StringifiedPlan, Subquery, SubqueryAlias, TableScan, ToStringifiedPlan, Union, Unnest, Values, Window, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index e0aae4cb7448..4b42702f24bf 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -221,10 +221,6 @@ pub enum LogicalPlan { /// Join two logical plans on one or more join columns. /// This is used to implement SQL `JOIN` Join(Join), - /// Apply Cross Join to two logical plans. - /// This is used to implement SQL `CROSS JOIN` - /// Deprecated: use [LogicalPlan::Join] instead with empty `on` / no filter - CrossJoin(CrossJoin), /// Repartitions the input based on a partitioning scheme. This is /// used to add parallelism and is sometimes referred to as an /// "exchange" operator in other systems @@ -312,7 +308,6 @@ impl LogicalPlan { LogicalPlan::Aggregate(Aggregate { schema, .. }) => schema, LogicalPlan::Sort(Sort { input, .. }) => input.schema(), LogicalPlan::Join(Join { schema, .. }) => schema, - LogicalPlan::CrossJoin(CrossJoin { schema, .. }) => schema, LogicalPlan::Repartition(Repartition { input, .. }) => input.schema(), LogicalPlan::Limit(Limit { input, .. }) => input.schema(), LogicalPlan::Statement(statement) => statement.schema(), @@ -345,8 +340,7 @@ impl LogicalPlan { | LogicalPlan::Projection(_) | LogicalPlan::Aggregate(_) | LogicalPlan::Unnest(_) - | LogicalPlan::Join(_) - | LogicalPlan::CrossJoin(_) => self + | LogicalPlan::Join(_) => self .inputs() .iter() .map(|input| input.schema().as_ref()) @@ -436,7 +430,6 @@ impl LogicalPlan { LogicalPlan::Aggregate(Aggregate { input, .. }) => vec![input], LogicalPlan::Sort(Sort { input, .. }) => vec![input], LogicalPlan::Join(Join { left, right, .. }) => vec![left, right], - LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => vec![left, right], LogicalPlan::Limit(Limit { input, .. }) => vec![input], LogicalPlan::Subquery(Subquery { subquery, .. }) => vec![subquery], LogicalPlan::SubqueryAlias(SubqueryAlias { input, .. }) => vec![input], @@ -542,13 +535,6 @@ impl LogicalPlan { JoinType::LeftSemi | JoinType::LeftAnti => left.head_output_expr(), JoinType::RightSemi | JoinType::RightAnti => right.head_output_expr(), }, - LogicalPlan::CrossJoin(cross) => { - if cross.left.schema().fields().is_empty() { - cross.right.head_output_expr() - } else { - cross.left.head_output_expr() - } - } LogicalPlan::RecursiveQuery(RecursiveQuery { static_term, .. }) => { static_term.head_output_expr() } @@ -674,20 +660,6 @@ impl LogicalPlan { null_equals_null, })) } - LogicalPlan::CrossJoin(CrossJoin { - left, - right, - schema: _, - }) => { - let join_schema = - build_join_schema(left.schema(), right.schema(), &JoinType::Inner)?; - - Ok(LogicalPlan::CrossJoin(CrossJoin { - left, - right, - schema: join_schema.into(), - })) - } LogicalPlan::Subquery(_) => Ok(self), LogicalPlan::SubqueryAlias(SubqueryAlias { input, @@ -938,11 +910,6 @@ impl LogicalPlan { null_equals_null: *null_equals_null, })) } - LogicalPlan::CrossJoin(_) => { - self.assert_no_expressions(expr)?; - let (left, right) = self.only_two_inputs(inputs)?; - LogicalPlanBuilder::from(left).cross_join(right)?.build() - } LogicalPlan::Subquery(Subquery { outer_ref_columns, .. }) => { @@ -1327,12 +1294,6 @@ impl LogicalPlan { JoinType::LeftSemi | JoinType::LeftAnti => left.max_rows(), JoinType::RightSemi | JoinType::RightAnti => right.max_rows(), }, - LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => { - match (left.max_rows(), right.max_rows()) { - (Some(left_max), Some(right_max)) => Some(left_max * right_max), - _ => None, - } - } LogicalPlan::Repartition(Repartition { input, .. }) => input.max_rows(), LogicalPlan::Union(Union { inputs, .. }) => inputs .iter() @@ -1893,9 +1854,6 @@ impl LogicalPlan { } } } - LogicalPlan::CrossJoin(_) => { - write!(f, "CrossJoin:") - } LogicalPlan::Repartition(Repartition { partitioning_scheme, .. @@ -2601,28 +2559,7 @@ impl TableScan { } } -/// Apply Cross Join to two logical plans -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct CrossJoin { - /// Left input - pub left: Arc, - /// Right input - pub right: Arc, - /// The output schema, containing fields from the left and right inputs - pub schema: DFSchemaRef, -} - -// Manual implementation needed because of `schema` field. Comparison excludes this field. -impl PartialOrd for CrossJoin { - fn partial_cmp(&self, other: &Self) -> Option { - match self.left.partial_cmp(&other.left) { - Some(Ordering::Equal) => self.right.partial_cmp(&other.right), - cmp => cmp, - } - } -} - -/// Repartition the plan based on a partitioning scheme. +// Repartition the plan based on a partitioning scheme. #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct Repartition { /// The incoming logical plan diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index b8d7043d7746..0658f7029740 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -37,11 +37,11 @@ //! * [`LogicalPlan::with_new_exprs`]: Create a new plan with different expressions //! * [`LogicalPlan::expressions`]: Return a copy of the plan's expressions use crate::{ - dml::CopyTo, Aggregate, Analyze, CreateMemoryTable, CreateView, CrossJoin, - DdlStatement, Distinct, DistinctOn, DmlStatement, Explain, Expr, Extension, Filter, - Join, Limit, LogicalPlan, Partitioning, Prepare, Projection, RecursiveQuery, - Repartition, Sort, Subquery, SubqueryAlias, TableScan, Union, Unnest, - UserDefinedLogicalNode, Values, Window, + dml::CopyTo, Aggregate, Analyze, CreateMemoryTable, CreateView, DdlStatement, + Distinct, DistinctOn, DmlStatement, Explain, Expr, Extension, Filter, Join, Limit, + LogicalPlan, Partitioning, Prepare, Projection, RecursiveQuery, Repartition, Sort, + Subquery, SubqueryAlias, TableScan, Union, Unnest, UserDefinedLogicalNode, Values, + Window, }; use std::ops::Deref; use std::sync::Arc; @@ -160,22 +160,6 @@ impl TreeNode for LogicalPlan { null_equals_null, }) }), - LogicalPlan::CrossJoin(CrossJoin { - left, - right, - schema, - }) => map_until_stop_and_collect!( - rewrite_arc(left, &mut f), - right, - rewrite_arc(right, &mut f) - )? - .update_data(|(left, right)| { - LogicalPlan::CrossJoin(CrossJoin { - left, - right, - schema, - }) - }), LogicalPlan::Limit(Limit { skip, fetch, input }) => rewrite_arc(input, f)? .update_data(|input| LogicalPlan::Limit(Limit { skip, fetch, input })), LogicalPlan::Subquery(Subquery { @@ -527,7 +511,6 @@ impl LogicalPlan { | LogicalPlan::Subquery(_) | LogicalPlan::SubqueryAlias(_) | LogicalPlan::Statement(_) - | LogicalPlan::CrossJoin(_) | LogicalPlan::Analyze(_) | LogicalPlan::Explain(_) | LogicalPlan::Union(_) @@ -758,7 +741,6 @@ impl LogicalPlan { | LogicalPlan::Subquery(_) | LogicalPlan::SubqueryAlias(_) | LogicalPlan::Statement(_) - | LogicalPlan::CrossJoin(_) | LogicalPlan::Analyze(_) | LogicalPlan::Explain(_) | LogicalPlan::Union(_) diff --git a/datafusion/optimizer/src/analyzer/subquery.rs b/datafusion/optimizer/src/analyzer/subquery.rs index aabc549de583..0a52685bd681 100644 --- a/datafusion/optimizer/src/analyzer/subquery.rs +++ b/datafusion/optimizer/src/analyzer/subquery.rs @@ -180,7 +180,6 @@ fn check_inner_plan( LogicalPlan::Projection(_) | LogicalPlan::Distinct(_) | LogicalPlan::Sort(_) - | LogicalPlan::CrossJoin(_) | LogicalPlan::Union(_) | LogicalPlan::TableScan(_) | LogicalPlan::EmptyRelation(_) diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 921011d33fc4..ee9ae9fb15a7 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -534,7 +534,6 @@ impl OptimizerRule for CommonSubexprEliminate { LogicalPlan::Window(window) => self.try_optimize_window(window, config)?, LogicalPlan::Aggregate(agg) => self.try_optimize_aggregate(agg, config)?, LogicalPlan::Join(_) - | LogicalPlan::CrossJoin(_) | LogicalPlan::Repartition(_) | LogicalPlan::Union(_) | LogicalPlan::TableScan(_) diff --git a/datafusion/optimizer/src/eliminate_cross_join.rs b/datafusion/optimizer/src/eliminate_cross_join.rs index 8a365fb389be..65ebac2106ad 100644 --- a/datafusion/optimizer/src/eliminate_cross_join.rs +++ b/datafusion/optimizer/src/eliminate_cross_join.rs @@ -98,7 +98,7 @@ impl OptimizerRule for EliminateCrossJoin { LogicalPlan::Join(Join { join_type: JoinType::Inner, .. - }) | LogicalPlan::CrossJoin(_) + }) ); if !rewriteable { @@ -241,20 +241,6 @@ fn flatten_join_inputs( all_filters, )?; } - LogicalPlan::CrossJoin(join) => { - flatten_join_inputs( - Arc::unwrap_or_clone(join.left), - possible_join_keys, - all_inputs, - all_filters, - )?; - flatten_join_inputs( - Arc::unwrap_or_clone(join.right), - possible_join_keys, - all_inputs, - all_filters, - )?; - } _ => { all_inputs.push(plan); } @@ -270,23 +256,18 @@ fn can_flatten_join_inputs(plan: &LogicalPlan) -> bool { // can only flatten inner / cross joins match plan { LogicalPlan::Join(join) if join.join_type == JoinType::Inner => {} - LogicalPlan::CrossJoin(_) => {} _ => return false, }; for child in plan.inputs() { - match child { - LogicalPlan::Join(Join { - join_type: JoinType::Inner, - .. - }) - | LogicalPlan::CrossJoin(_) => { - if !can_flatten_join_inputs(child) { - return false; - } + if let LogicalPlan::Join(Join { + join_type: JoinType::Inner, + .. + }) = child + { + if !can_flatten_join_inputs(child) { + return false; } - // the child is not a join/cross join - _ => (), } } true diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index b5d581f3919f..42eff7100fbe 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -367,17 +367,6 @@ fn optimize_projections( right_indices.with_projection_beneficial(), ] } - LogicalPlan::CrossJoin(cross_join) => { - let left_len = cross_join.left.schema().fields().len(); - let (left_indices, right_indices) = - split_join_requirements(left_len, indices, &JoinType::Inner); - // Joins benefit from "small" input tables (lower memory usage). - // Therefore, each child benefits from projection: - vec![ - left_indices.with_projection_beneficial(), - right_indices.with_projection_beneficial(), - ] - } // these nodes are explicitly rewritten in the match statement above LogicalPlan::Projection(_) | LogicalPlan::Aggregate(_) diff --git a/datafusion/optimizer/src/propagate_empty_relation.rs b/datafusion/optimizer/src/propagate_empty_relation.rs index b5e1077ee5be..d26df073dc6f 100644 --- a/datafusion/optimizer/src/propagate_empty_relation.rs +++ b/datafusion/optimizer/src/propagate_empty_relation.rs @@ -72,19 +72,6 @@ impl OptimizerRule for PropagateEmptyRelation { } Ok(Transformed::no(plan)) } - LogicalPlan::CrossJoin(ref join) => { - let (left_empty, right_empty) = binary_plan_children_is_empty(&plan)?; - if left_empty || right_empty { - return Ok(Transformed::yes(LogicalPlan::EmptyRelation( - EmptyRelation { - produce_one_row: false, - schema: Arc::clone(plan.schema()), - }, - ))); - } - Ok(Transformed::no(LogicalPlan::CrossJoin(join.clone()))) - } - LogicalPlan::Join(ref join) => { // TODO: For Join, more join type need to be careful: // For LeftOut/Full Join, if the right side is empty, the Join can be eliminated with a Projection with left side diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index ac81f3efaa11..a6c0a7310610 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -24,19 +24,15 @@ use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; use datafusion_common::{ - internal_err, plan_err, qualified_name, Column, DFSchema, DFSchemaRef, - JoinConstraint, Result, + internal_err, plan_err, qualified_name, Column, DFSchema, Result, }; use datafusion_expr::expr_rewriter::replace_col; -use datafusion_expr::logical_plan::{ - CrossJoin, Join, JoinType, LogicalPlan, TableScan, Union, -}; +use datafusion_expr::logical_plan::{Join, JoinType, LogicalPlan, TableScan, Union}; use datafusion_expr::utils::{ conjunction, expr_to_columns, split_conjunction, split_conjunction_owned, }; use datafusion_expr::{ - and, build_join_schema, or, BinaryExpr, Expr, Filter, LogicalPlanBuilder, Operator, - Projection, TableProviderFilterPushDown, + and, or, BinaryExpr, Expr, Filter, Operator, Projection, TableProviderFilterPushDown, }; use crate::optimizer::ApplyOrder; @@ -867,12 +863,6 @@ impl OptimizerRule for PushDownFilter { }) } LogicalPlan::Join(join) => push_down_join(join, Some(&filter.predicate)), - LogicalPlan::CrossJoin(cross_join) => { - let predicates = split_conjunction_owned(filter.predicate); - let join = convert_cross_join_to_inner_join(cross_join)?; - let plan = push_down_all_join(predicates, vec![], join, vec![])?; - convert_to_cross_join_if_beneficial(plan.data) - } LogicalPlan::TableScan(scan) => { let filter_predicates = split_conjunction(&filter.predicate); let results = scan @@ -1114,48 +1104,6 @@ impl PushDownFilter { } } -/// Converts the given cross join to an inner join with an empty equality -/// predicate and an empty filter condition. -fn convert_cross_join_to_inner_join(cross_join: CrossJoin) -> Result { - let CrossJoin { left, right, .. } = cross_join; - let join_schema = build_join_schema(left.schema(), right.schema(), &JoinType::Inner)?; - Ok(Join { - left, - right, - join_type: JoinType::Inner, - join_constraint: JoinConstraint::On, - on: vec![], - filter: None, - schema: DFSchemaRef::new(join_schema), - null_equals_null: false, - }) -} - -/// Converts the given inner join with an empty equality predicate and an -/// empty filter condition to a cross join. -fn convert_to_cross_join_if_beneficial( - plan: LogicalPlan, -) -> Result> { - match plan { - // Can be converted back to cross join - LogicalPlan::Join(join) if join.on.is_empty() && join.filter.is_none() => { - LogicalPlanBuilder::from(Arc::unwrap_or_clone(join.left)) - .cross_join(Arc::unwrap_or_clone(join.right))? - .build() - .map(Transformed::yes) - } - LogicalPlan::Filter(filter) => { - convert_to_cross_join_if_beneficial(Arc::unwrap_or_clone(filter.input))? - .transform_data(|child_plan| { - Filter::try_new(filter.predicate, Arc::new(child_plan)) - .map(LogicalPlan::Filter) - .map(Transformed::yes) - }) - } - plan => Ok(Transformed::no(plan)), - } -} - /// replaces columns by its name on the projection. pub fn replace_cols_by_name( e: Expr, @@ -1203,13 +1151,13 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use async_trait::async_trait; - use datafusion_common::ScalarValue; + use datafusion_common::{DFSchemaRef, ScalarValue}; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{ - col, in_list, in_subquery, lit, ColumnarValue, Extension, ScalarUDF, - ScalarUDFImpl, Signature, TableSource, TableType, UserDefinedLogicalNodeCore, - Volatility, + col, in_list, in_subquery, lit, ColumnarValue, Extension, LogicalPlanBuilder, + ScalarUDF, ScalarUDFImpl, Signature, TableSource, TableType, + UserDefinedLogicalNodeCore, Volatility, }; use crate::optimizer::Optimizer; diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index bf5ce0531e06..ec7a0a1364b6 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -118,13 +118,6 @@ impl OptimizerRule for PushDownLimit { transformed_limit(skip, fetch, LogicalPlan::Union(union)) } - LogicalPlan::CrossJoin(mut cross_join) => { - // push limit to both inputs - cross_join.left = make_arc_limit(0, fetch + skip, cross_join.left); - cross_join.right = make_arc_limit(0, fetch + skip, cross_join.right); - transformed_limit(skip, fetch, LogicalPlan::CrossJoin(cross_join)) - } - LogicalPlan::Join(join) => Ok(push_down_join(join, fetch + skip) .update_data(|join| { make_limit(skip, fetch, Arc::new(LogicalPlan::Join(join))) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 73df506397b1..d80c6b716537 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -61,9 +61,9 @@ use datafusion_expr::{ dml, logical_plan::{ builder::project, Aggregate, CreateCatalog, CreateCatalogSchema, - CreateExternalTable, CreateView, CrossJoin, DdlStatement, Distinct, - EmptyRelation, Extension, Join, JoinConstraint, Prepare, Projection, Repartition, - Sort, SubqueryAlias, TableScan, Values, Window, + CreateExternalTable, CreateView, DdlStatement, Distinct, EmptyRelation, + Extension, Join, JoinConstraint, Prepare, Projection, Repartition, Sort, + SubqueryAlias, TableScan, Values, Window, }, DistinctOn, DropView, Expr, LogicalPlan, LogicalPlanBuilder, ScalarUDF, SortExpr, WindowUDF, @@ -1503,24 +1503,6 @@ impl AsLogicalPlan for LogicalPlanNode { )), }) } - LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => { - let left = protobuf::LogicalPlanNode::try_from_logical_plan( - left.as_ref(), - extension_codec, - )?; - let right = protobuf::LogicalPlanNode::try_from_logical_plan( - right.as_ref(), - extension_codec, - )?; - Ok(protobuf::LogicalPlanNode { - logical_plan_type: Some(LogicalPlanType::CrossJoin(Box::new( - protobuf::CrossJoinNode { - left: Some(Box::new(left)), - right: Some(Box::new(right)), - }, - ))), - }) - } LogicalPlan::Extension(extension) => { let mut buf: Vec = vec![]; extension_codec.try_encode(extension, &mut buf)?; diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index 0147a607567b..695027374fa0 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -96,7 +96,6 @@ impl Unparser<'_> { | LogicalPlan::Aggregate(_) | LogicalPlan::Sort(_) | LogicalPlan::Join(_) - | LogicalPlan::CrossJoin(_) | LogicalPlan::Repartition(_) | LogicalPlan::Union(_) | LogicalPlan::TableScan(_) @@ -497,43 +496,6 @@ impl Unparser<'_> { Ok(()) } - LogicalPlan::CrossJoin(cross_join) => { - // Cross joins are the same as unconditional inner joins - let mut right_relation = RelationBuilder::default(); - - self.select_to_sql_recursively( - cross_join.left.as_ref(), - query, - select, - relation, - )?; - self.select_to_sql_recursively( - cross_join.right.as_ref(), - query, - select, - &mut right_relation, - )?; - - let Ok(Some(relation)) = right_relation.build() else { - return internal_err!("Failed to build right relation"); - }; - - let ast_join = ast::Join { - relation, - global: false, - join_operator: self.join_operator_to_sql( - JoinType::Inner, - ast::JoinConstraint::On(ast::Expr::Value(ast::Value::Boolean( - true, - ))), - ), - }; - let mut from = select.pop_from().unwrap(); - from.push_join(ast_join); - select.push_from(from); - - Ok(()) - } LogicalPlan::SubqueryAlias(plan_alias) => { let (plan, mut columns) = subquery_alias_inner_query_and_columns(plan_alias); diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index bb50c4b9610f..4105dc1876db 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -24,7 +24,7 @@ use substrait::proto::expression_reference::ExprType; use arrow_buffer::ToByteSlice; use datafusion::arrow::datatypes::{Field, IntervalUnit}; use datafusion::logical_expr::{ - CrossJoin, Distinct, FetchType, Like, Partitioning, SkipType, WindowFrameUnits, + Distinct, FetchType, Like, Partitioning, SkipType, WindowFrameUnits, }; use datafusion::{ arrow::datatypes::{DataType, TimeUnit}, @@ -67,7 +67,7 @@ use substrait::proto::read_rel::VirtualTable; use substrait::proto::rel_common::EmitKind; use substrait::proto::rel_common::EmitKind::Emit; use substrait::proto::{ - rel_common, CrossRel, ExchangeRel, ExpressionReference, ExtendedExpression, RelCommon, + rel_common, ExchangeRel, ExpressionReference, ExtendedExpression, RelCommon, }; use substrait::{ proto::{ @@ -476,23 +476,6 @@ pub fn to_substrait_rel( }))), })) } - LogicalPlan::CrossJoin(cross_join) => { - let CrossJoin { - left, - right, - schema: _, - } = cross_join; - let left = to_substrait_rel(left.as_ref(), ctx, extensions)?; - let right = to_substrait_rel(right.as_ref(), ctx, extensions)?; - Ok(Box::new(Rel { - rel_type: Some(RelType::Cross(Box::new(CrossRel { - common: None, - left: Some(left), - right: Some(right), - advanced_extension: None, - }))), - })) - } LogicalPlan::SubqueryAlias(alias) => { // Do nothing if encounters SubqueryAlias // since there is no corresponding relation type in Substrait From ac827abe1b66b1dfa02ce65ae857477f68667843 Mon Sep 17 00:00:00 2001 From: Jonathan Chen Date: Thu, 24 Oct 2024 07:26:37 -0400 Subject: [PATCH 04/27] feat: Migrate Map Functions (#13047) * add page * small fixes * delete md * Migrate map functions --- datafusion/functions-nested/src/map.rs | 69 ++++++++- .../functions-nested/src/map_extract.rs | 49 +++++- datafusion/functions-nested/src/map_keys.rs | 41 ++++- datafusion/functions-nested/src/map_values.rs | 41 ++++- dev/update_function_docs.sh | 1 - .../source/user-guide/sql/scalar_functions.md | 145 ------------------ .../user-guide/sql/scalar_functions_new.md | 144 +++++++++++++++++ 7 files changed, 334 insertions(+), 156 deletions(-) diff --git a/datafusion/functions-nested/src/map.rs b/datafusion/functions-nested/src/map.rs index 29afe4a7f3be..d7dce3bacbe1 100644 --- a/datafusion/functions-nested/src/map.rs +++ b/datafusion/functions-nested/src/map.rs @@ -17,7 +17,7 @@ use std::any::Any; use std::collections::{HashSet, VecDeque}; -use std::sync::Arc; +use std::sync::{Arc, OnceLock}; use arrow::array::ArrayData; use arrow_array::{Array, ArrayRef, MapArray, OffsetSizeTrait, StructArray}; @@ -27,7 +27,10 @@ use arrow_schema::{DataType, Field, SchemaBuilder}; use datafusion_common::utils::{fixed_size_list_to_arrays, list_to_arrays}; use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::expr::ScalarFunction; -use datafusion_expr::{ColumnarValue, Expr, ScalarUDFImpl, Signature, Volatility}; +use datafusion_expr::scalar_doc_sections::DOC_SECTION_MAP; +use datafusion_expr::{ + ColumnarValue, Documentation, Expr, ScalarUDFImpl, Signature, Volatility, +}; use crate::make_array::make_array; @@ -238,7 +241,69 @@ impl ScalarUDFImpl for MapFunc { fn invoke(&self, args: &[ColumnarValue]) -> Result { make_map_batch(args) } + + fn documentation(&self) -> Option<&Documentation> { + Some(get_map_doc()) + } } + +static DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_map_doc() -> &'static Documentation { + DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_MAP) + .with_description( + "Returns an Arrow map with the specified key-value pairs.\n\n\ + The `make_map` function creates a map from two lists: one for keys and one for values. Each key must be unique and non-null." + ) + .with_syntax_example( + "map(key, value)\nmap(key: value)\nmake_map(['key1', 'key2'], ['value1', 'value2'])" + ) + .with_sql_example( + r#"```sql + -- Using map function + SELECT MAP('type', 'test'); + ---- + {type: test} + + SELECT MAP(['POST', 'HEAD', 'PATCH'], [41, 33, null]); + ---- + {POST: 41, HEAD: 33, PATCH: } + + SELECT MAP([[1,2], [3,4]], ['a', 'b']); + ---- + {[1, 2]: a, [3, 4]: b} + + SELECT MAP { 'a': 1, 'b': 2 }; + ---- + {a: 1, b: 2} + + -- Using make_map function + SELECT MAKE_MAP(['POST', 'HEAD'], [41, 33]); + ---- + {POST: 41, HEAD: 33} + + SELECT MAKE_MAP(['key1', 'key2'], ['value1', null]); + ---- + {key1: value1, key2: } + ```"# + ) + .with_argument( + "key", + "For `map`: Expression to be used for key. Can be a constant, column, function, or any combination of arithmetic or string operators.\n\ + For `make_map`: The list of keys to be used in the map. Each key must be unique and non-null." + ) + .with_argument( + "value", + "For `map`: Expression to be used for value. Can be a constant, column, function, or any combination of arithmetic or string operators.\n\ + For `make_map`: The list of values to be mapped to the corresponding keys." + ) + .build() + .unwrap() + }) +} + fn get_element_type(data_type: &DataType) -> Result<&DataType> { match data_type { DataType::List(element) => Ok(element.data_type()), diff --git a/datafusion/functions-nested/src/map_extract.rs b/datafusion/functions-nested/src/map_extract.rs index 9f0c4ad29c60..d2bb6595fe76 100644 --- a/datafusion/functions-nested/src/map_extract.rs +++ b/datafusion/functions-nested/src/map_extract.rs @@ -26,9 +26,12 @@ use arrow_buffer::OffsetBuffer; use arrow_schema::Field; use datafusion_common::{cast::as_map_array, exec_err, Result}; -use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; +use datafusion_expr::scalar_doc_sections::DOC_SECTION_MAP; +use datafusion_expr::{ + ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, +}; use std::any::Any; -use std::sync::Arc; +use std::sync::{Arc, OnceLock}; use std::vec; use crate::utils::{get_map_entry_field, make_scalar_function}; @@ -101,6 +104,48 @@ impl ScalarUDFImpl for MapExtract { field.first().unwrap().data_type().clone(), ]) } + + fn documentation(&self) -> Option<&Documentation> { + Some(get_map_extract_doc()) + } +} + +static DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_map_extract_doc() -> &'static Documentation { + DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_MAP) + .with_description( + "Returns a list containing the value for the given key or an empty list if the key is not present in the map.", + ) + .with_syntax_example("map_extract(map, key)") + .with_sql_example( + r#"```sql +SELECT map_extract(MAP {'a': 1, 'b': NULL, 'c': 3}, 'a'); +---- +[1] + +SELECT map_extract(MAP {1: 'one', 2: 'two'}, 2); +---- +['two'] + +SELECT map_extract(MAP {'x': 10, 'y': NULL, 'z': 30}, 'y'); +---- +[] +```"#, + ) + .with_argument( + "map", + "Map expression. Can be a constant, column, or function, and any combination of map operators.", + ) + .with_argument( + "key", + "Key to extract from the map. Can be a constant, column, or function, any combination of arithmetic or string operators, or a named expression of the previously listed.", + ) + .build() + .unwrap() + }) } fn general_map_extract_inner( diff --git a/datafusion/functions-nested/src/map_keys.rs b/datafusion/functions-nested/src/map_keys.rs index 0b1cebb27c86..f28de1c3b2c7 100644 --- a/datafusion/functions-nested/src/map_keys.rs +++ b/datafusion/functions-nested/src/map_keys.rs @@ -21,12 +21,13 @@ use crate::utils::{get_map_entry_field, make_scalar_function}; use arrow_array::{Array, ArrayRef, ListArray}; use arrow_schema::{DataType, Field}; use datafusion_common::{cast::as_map_array, exec_err, Result}; +use datafusion_expr::scalar_doc_sections::DOC_SECTION_MAP; use datafusion_expr::{ - ArrayFunctionSignature, ColumnarValue, ScalarUDFImpl, Signature, TypeSignature, - Volatility, + ArrayFunctionSignature, ColumnarValue, Documentation, ScalarUDFImpl, Signature, + TypeSignature, Volatility, }; use std::any::Any; -use std::sync::Arc; +use std::sync::{Arc, OnceLock}; make_udf_expr_and_func!( MapKeysFunc, @@ -81,6 +82,40 @@ impl ScalarUDFImpl for MapKeysFunc { fn invoke(&self, args: &[ColumnarValue]) -> datafusion_common::Result { make_scalar_function(map_keys_inner)(args) } + + fn documentation(&self) -> Option<&Documentation> { + Some(get_map_keys_doc()) + } +} + +static DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_map_keys_doc() -> &'static Documentation { + DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_MAP) + .with_description( + "Returns a list of all keys in the map." + ) + .with_syntax_example("map_keys(map)") + .with_sql_example( + r#"```sql +SELECT map_keys(MAP {'a': 1, 'b': NULL, 'c': 3}); +---- +[a, b, c] + +SELECT map_keys(map([100, 5], [42, 43])); +---- +[100, 5] +```"#, + ) + .with_argument( + "map", + "Map expression. Can be a constant, column, or function, and any combination of map operators." + ) + .build() + .unwrap() + }) } fn map_keys_inner(args: &[ArrayRef]) -> Result { diff --git a/datafusion/functions-nested/src/map_values.rs b/datafusion/functions-nested/src/map_values.rs index 58c0d74eed5f..2b19d9fbbc76 100644 --- a/datafusion/functions-nested/src/map_values.rs +++ b/datafusion/functions-nested/src/map_values.rs @@ -21,12 +21,13 @@ use crate::utils::{get_map_entry_field, make_scalar_function}; use arrow_array::{Array, ArrayRef, ListArray}; use arrow_schema::{DataType, Field}; use datafusion_common::{cast::as_map_array, exec_err, Result}; +use datafusion_expr::scalar_doc_sections::DOC_SECTION_MAP; use datafusion_expr::{ - ArrayFunctionSignature, ColumnarValue, ScalarUDFImpl, Signature, TypeSignature, - Volatility, + ArrayFunctionSignature, ColumnarValue, Documentation, ScalarUDFImpl, Signature, + TypeSignature, Volatility, }; use std::any::Any; -use std::sync::Arc; +use std::sync::{Arc, OnceLock}; make_udf_expr_and_func!( MapValuesFunc, @@ -81,6 +82,40 @@ impl ScalarUDFImpl for MapValuesFunc { fn invoke(&self, args: &[ColumnarValue]) -> datafusion_common::Result { make_scalar_function(map_values_inner)(args) } + + fn documentation(&self) -> Option<&Documentation> { + Some(get_map_values_doc()) + } +} + +static DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_map_values_doc() -> &'static Documentation { + DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_MAP) + .with_description( + "Returns a list of all values in the map." + ) + .with_syntax_example("map_values(map)") + .with_sql_example( + r#"```sql +SELECT map_values(MAP {'a': 1, 'b': NULL, 'c': 3}); +---- +[1, , 3] + +SELECT map_values(map([100, 5], [42, 43])); +---- +[42, 43] +```"#, + ) + .with_argument( + "map", + "Map expression. Can be a constant, column, or function, and any combination of map operators." + ) + .build() + .unwrap() + }) } fn map_values_inner(args: &[ArrayRef]) -> Result { diff --git a/dev/update_function_docs.sh b/dev/update_function_docs.sh index f1f26c8b2f58..13bc22afcc13 100755 --- a/dev/update_function_docs.sh +++ b/dev/update_function_docs.sh @@ -297,4 +297,3 @@ echo "Running prettier" npx prettier@2.3.2 --write "$TARGET_FILE" echo "'$TARGET_FILE' successfully updated!" - diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 547ea108080e..203411428777 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -199,151 +199,6 @@ Unwraps struct fields into columns. +-----------------------+-----------------------+ ``` -## Map Functions - -- [map](#map) -- [make_map](#make_map) -- [map_extract](#map_extract) -- [map_keys](#map_keys) -- [map_values](#map_values) - -### `map` - -Returns an Arrow map with the specified key-value pairs. - -``` -map(key, value) -map(key: value) -``` - -#### Arguments - -- **key**: Expression to be used for key. - Can be a constant, column, or function, any combination of arithmetic or - string operators, or a named expression of previous listed. -- **value**: Expression to be used for value. - Can be a constant, column, or function, any combination of arithmetic or - string operators, or a named expression of previous listed. - -#### Example - -``` -SELECT MAP(['POST', 'HEAD', 'PATCH'], [41, 33, null]); ----- -{POST: 41, HEAD: 33, PATCH: } - -SELECT MAP([[1,2], [3,4]], ['a', 'b']); ----- -{[1, 2]: a, [3, 4]: b} - -SELECT MAP { 'a': 1, 'b': 2 }; ----- -{a: 1, b: 2} -``` - -### `make_map` - -Returns an Arrow map with the specified key-value pairs. - -``` -make_map(key_1, value_1, ..., key_n, value_n) -``` - -#### Arguments - -- **key_n**: Expression to be used for key. - Can be a constant, column, or function, any combination of arithmetic or - string operators, or a named expression of previous listed. -- **value_n**: Expression to be used for value. - Can be a constant, column, or function, any combination of arithmetic or - string operators, or a named expression of previous listed. - -#### Example - -``` -SELECT MAKE_MAP('POST', 41, 'HEAD', 33, 'PATCH', null); ----- -{POST: 41, HEAD: 33, PATCH: } -``` - -### `map_extract` - -Return a list containing the value for a given key or an empty list if the key is not contained in the map. - -``` -map_extract(map, key) -``` - -#### Arguments - -- `map`: Map expression. - Can be a constant, column, or function, and any combination of map operators. -- `key`: Key to extract from the map. - Can be a constant, column, or function, any combination of arithmetic or - string operators, or a named expression of previous listed. - -#### Example - -``` -SELECT map_extract(MAP {'a': 1, 'b': NULL, 'c': 3}, 'a'); ----- -[1] -``` - -#### Aliases - -- element_at - -### `map_keys` - -Return a list of all keys in the map. - -``` -map_keys(map) -``` - -#### Arguments - -- `map`: Map expression. - Can be a constant, column, or function, and any combination of map operators. - -#### Example - -``` -SELECT map_keys(MAP {'a': 1, 'b': NULL, 'c': 3}); ----- -[a, b, c] - -select map_keys(map([100, 5], [42,43])); ----- -[100, 5] -``` - -### `map_values` - -Return a list of all values in the map. - -``` -map_values(map) -``` - -#### Arguments - -- `map`: Map expression. - Can be a constant, column, or function, and any combination of map operators. - -#### Example - -``` -SELECT map_values(MAP {'a': 1, 'b': NULL, 'c': 3}); ----- -[1, , 3] - -select map_values(map([100, 5], [42,43])); ----- -[42, 43] -``` - ## Other Functions See the new documentation [`here`](https://datafusion.apache.org/user-guide/sql/scalar_functions_new.html) diff --git a/docs/source/user-guide/sql/scalar_functions_new.md b/docs/source/user-guide/sql/scalar_functions_new.md index 1f4ec1c27858..7d0280dbc28f 100644 --- a/docs/source/user-guide/sql/scalar_functions_new.md +++ b/docs/source/user-guide/sql/scalar_functions_new.md @@ -3898,6 +3898,150 @@ select struct(a as field_a, b) from t; - row +## Map Functions + +- [element_at](#element_at) +- [map](#map) +- [map_extract](#map_extract) +- [map_keys](#map_keys) +- [map_values](#map_values) + +### `element_at` + +_Alias of [map_extract](#map_extract)._ + +### `map` + +Returns an Arrow map with the specified key-value pairs. + +The `make_map` function creates a map from two lists: one for keys and one for values. Each key must be unique and non-null. + +``` +map(key, value) +map(key: value) +make_map(['key1', 'key2'], ['value1', 'value2']) +``` + +#### Arguments + +- **key**: For `map`: Expression to be used for key. Can be a constant, column, function, or any combination of arithmetic or string operators. + For `make_map`: The list of keys to be used in the map. Each key must be unique and non-null. +- **value**: For `map`: Expression to be used for value. Can be a constant, column, function, or any combination of arithmetic or string operators. + For `make_map`: The list of values to be mapped to the corresponding keys. + +#### Example + +````sql + -- Using map function + SELECT MAP('type', 'test'); + ---- + {type: test} + + SELECT MAP(['POST', 'HEAD', 'PATCH'], [41, 33, null]); + ---- + {POST: 41, HEAD: 33, PATCH: } + + SELECT MAP([[1,2], [3,4]], ['a', 'b']); + ---- + {[1, 2]: a, [3, 4]: b} + + SELECT MAP { 'a': 1, 'b': 2 }; + ---- + {a: 1, b: 2} + + -- Using make_map function + SELECT MAKE_MAP(['POST', 'HEAD'], [41, 33]); + ---- + {POST: 41, HEAD: 33} + + SELECT MAKE_MAP(['key1', 'key2'], ['value1', null]); + ---- + {key1: value1, key2: } + ``` + + +### `map_extract` + +Returns a list containing the value for the given key or an empty list if the key is not present in the map. + +```` + +map_extract(map, key) + +```` +#### Arguments + +- **map**: Map expression. Can be a constant, column, or function, and any combination of map operators. +- **key**: Key to extract from the map. Can be a constant, column, or function, any combination of arithmetic or string operators, or a named expression of the previously listed. + +#### Example + +```sql +SELECT map_extract(MAP {'a': 1, 'b': NULL, 'c': 3}, 'a'); +---- +[1] + +SELECT map_extract(MAP {1: 'one', 2: 'two'}, 2); +---- +['two'] + +SELECT map_extract(MAP {'x': 10, 'y': NULL, 'z': 30}, 'y'); +---- +[] +```` + +#### Aliases + +- element_at + +### `map_keys` + +Returns a list of all keys in the map. + +``` +map_keys(map) +``` + +#### Arguments + +- **map**: Map expression. Can be a constant, column, or function, and any combination of map operators. + +#### Example + +```sql +SELECT map_keys(MAP {'a': 1, 'b': NULL, 'c': 3}); +---- +[a, b, c] + +SELECT map_keys(map([100, 5], [42, 43])); +---- +[100, 5] +``` + +### `map_values` + +Returns a list of all values in the map. + +``` +map_values(map) +``` + +#### Arguments + +- **map**: Map expression. Can be a constant, column, or function, and any combination of map operators. + +#### Example + +```sql +SELECT map_values(MAP {'a': 1, 'b': NULL, 'c': 3}); +---- +[1, , 3] + +SELECT map_values(map([100, 5], [42, 43])); +---- +[42, 43] +``` + ## Hashing Functions - [digest](#digest) From 307c1ea2ef2323aa347be029039b9daf9b419645 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 24 Oct 2024 11:04:26 -0400 Subject: [PATCH 05/27] Minor: Add documentation for `cot` (#13069) * Add documentation for `cot` * fmt --- datafusion/functions/src/math/cot.rs | 26 ++++++++++++++++--- .../user-guide/sql/scalar_functions_new.md | 13 ++++++++++ 2 files changed, 35 insertions(+), 4 deletions(-) diff --git a/datafusion/functions/src/math/cot.rs b/datafusion/functions/src/math/cot.rs index f039767536fa..eded50a20d8d 100644 --- a/datafusion/functions/src/math/cot.rs +++ b/datafusion/functions/src/math/cot.rs @@ -16,18 +16,18 @@ // under the License. use std::any::Any; -use std::sync::Arc; +use std::sync::{Arc, OnceLock}; use arrow::array::{ArrayRef, AsArray}; use arrow::datatypes::DataType::{Float32, Float64}; use arrow::datatypes::{DataType, Float32Type, Float64Type}; +use crate::utils::make_scalar_function; use datafusion_common::{exec_err, Result}; -use datafusion_expr::ColumnarValue; +use datafusion_expr::scalar_doc_sections::DOC_SECTION_MATH; +use datafusion_expr::{ColumnarValue, Documentation}; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; -use crate::utils::make_scalar_function; - #[derive(Debug)] pub struct CotFunc { signature: Signature, @@ -39,6 +39,20 @@ impl Default for CotFunc { } } +static DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_cot_doc() -> &'static Documentation { + DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_MATH) + .with_description("Returns the cotangent of a number.") + .with_syntax_example(r#"cot(numeric_expression)"#) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() + .unwrap() + }) +} + impl CotFunc { pub fn new() -> Self { use DataType::*; @@ -77,6 +91,10 @@ impl ScalarUDFImpl for CotFunc { } } + fn documentation(&self) -> Option<&Documentation> { + Some(get_cot_doc()) + } + fn invoke(&self, args: &[ColumnarValue]) -> Result { make_scalar_function(cot, vec![])(args) } diff --git a/docs/source/user-guide/sql/scalar_functions_new.md b/docs/source/user-guide/sql/scalar_functions_new.md index 7d0280dbc28f..55e61984d7f8 100644 --- a/docs/source/user-guide/sql/scalar_functions_new.md +++ b/docs/source/user-guide/sql/scalar_functions_new.md @@ -47,6 +47,7 @@ the rest of the documentation. - [ceil](#ceil) - [cos](#cos) - [cosh](#cosh) +- [cot](#cot) - [degrees](#degrees) - [exp](#exp) - [factorial](#factorial) @@ -221,6 +222,18 @@ cosh(numeric_expression) - **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. +### `cot` + +Returns the cotangent of a number. + +``` +cot(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + ### `degrees` Converts radians to degrees. From 631408baefa79d80f2f7e59e8a6e22714312cc3e Mon Sep 17 00:00:00 2001 From: Oleks V Date: Thu, 24 Oct 2024 09:25:03 -0700 Subject: [PATCH 06/27] Documentation: Add API deprecation policy (#13083) * Documentation: Add API deprecation policy --- README.md | 5 +++ docs/source/library-user-guide/api-health.md | 37 ++++++++++++++++++++ 2 files changed, 42 insertions(+) create mode 100644 docs/source/library-user-guide/api-health.md diff --git a/README.md b/README.md index 30505d7ca132..bbbdf7133518 100644 --- a/README.md +++ b/README.md @@ -134,3 +134,8 @@ For example, given the releases `1.78.0`, `1.79.0`, `1.80.0`, `1.80.1` and `1.81 If a hotfix is released for the minimum supported Rust version (MSRV), the MSRV will be the minor version with all hotfixes, even if it surpasses the four-month window. We enforce this policy using a [MSRV CI Check](https://github.com/search?q=repo%3Aapache%2Fdatafusion+rust-version+language%3ATOML+path%3A%2F%5ECargo.toml%2F&type=code) + +## DataFusion API evolution policy + +Public methods in Apache DataFusion are subject to evolve as part of the API lifecycle. +Deprecated methods will be phased out in accordance with the [policy](docs/source/library-user-guide/api-health.md), ensuring the API is stable and healthy. diff --git a/docs/source/library-user-guide/api-health.md b/docs/source/library-user-guide/api-health.md new file mode 100644 index 000000000000..943a370e8172 --- /dev/null +++ b/docs/source/library-user-guide/api-health.md @@ -0,0 +1,37 @@ + + +# API health policy + +To maintain API health, developers must track and properly deprecate outdated methods. +When deprecating a method: + +- clearly mark the API as deprecated and specify the exact DataFusion version in which it was deprecated. +- concisely describe the preferred API, if relevant + +API deprecation example: + +```rust + #[deprecated(since = "41.0.0", note = "Use SessionStateBuilder")] + pub fn new_with_config_rt(config: SessionConfig, runtime: Arc) -> Self +``` + +Deprecated methods will remain in the codebase for a period of 6 major versions or 6 months, whichever is longer, to provide users ample time to transition away from them. + +Please refer to [DataFusion releases](https://crates.io/crates/datafusion/versions) to plan ahead API migration From 1b14655148377fac317153b5b5f14f4256a5d375 Mon Sep 17 00:00:00 2001 From: Jonathan Chen Date: Thu, 24 Oct 2024 14:41:39 -0400 Subject: [PATCH 07/27] changed doc instance (#13097) --- datafusion/functions-nested/src/range.rs | 4 ++- .../user-guide/sql/scalar_functions_new.md | 29 +++++++------------ 2 files changed, 14 insertions(+), 19 deletions(-) diff --git a/datafusion/functions-nested/src/range.rs b/datafusion/functions-nested/src/range.rs index 2346b4d5b43f..ddc56b1e4ee8 100644 --- a/datafusion/functions-nested/src/range.rs +++ b/datafusion/functions-nested/src/range.rs @@ -283,8 +283,10 @@ impl ScalarUDFImpl for GenSeries { } } +static GENERATE_SERIES_DOCUMENTATION: OnceLock = OnceLock::new(); + fn get_generate_series_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { + GENERATE_SERIES_DOCUMENTATION.get_or_init(|| { Documentation::builder() .with_doc_section(DOC_SECTION_ARRAY) .with_description( diff --git a/docs/source/user-guide/sql/scalar_functions_new.md b/docs/source/user-guide/sql/scalar_functions_new.md index 55e61984d7f8..c15821ac89a3 100644 --- a/docs/source/user-guide/sql/scalar_functions_new.md +++ b/docs/source/user-guide/sql/scalar_functions_new.md @@ -3530,34 +3530,27 @@ flatten(array) ### `generate_series` -Returns an Arrow array between start and stop with step. The range start..end contains all values with start <= x < end. It is empty if start >= end. Step cannot be 0. +Similar to the range function, but it includes the upper bound. ``` -range(start, stop, step) +generate_series(start, stop, step) ``` #### Arguments -- **start**: Start of the range. Ints, timestamps, dates or string types that can be coerced to Date32 are supported. -- **end**: End of the range (not included). Type must be the same as start. -- **step**: Increase by step (cannot be 0). Steps less than a day are supported only for timestamp ranges. +- **start**: start of the series. Ints, timestamps, dates or string types that can be coerced to Date32 are supported. +- **end**: end of the series (included). Type must be the same as start. +- **step**: increase by step (can not be 0). Steps less than a day are supported only for timestamp ranges. #### Example ```sql -> select range(2, 10, 3); -+-----------------------------------+ -| range(Int64(2),Int64(10),Int64(3))| -+-----------------------------------+ -| [2, 5, 8] | -+-----------------------------------+ - -> select range(DATE '1992-09-01', DATE '1993-03-01', INTERVAL '1' MONTH); -+--------------------------------------------------------------+ -| range(DATE '1992-09-01', DATE '1993-03-01', INTERVAL '1' MONTH) | -+--------------------------------------------------------------+ -| [1992-09-01, 1992-10-01, 1992-11-01, 1992-12-01, 1993-01-01, 1993-02-01] | -+--------------------------------------------------------------+ +> select generate_series(1,3); ++------------------------------------+ +| generate_series(Int64(1),Int64(3)) | ++------------------------------------+ +| [1, 2, 3] | ++------------------------------------+ ``` ### `list_any_value` From 31701b8dc9c6486856c06a29a32107d9f4549cec Mon Sep 17 00:00:00 2001 From: Max Norfolk <66913041+mnorfolk03@users.noreply.github.com> Date: Thu, 24 Oct 2024 15:01:08 -0400 Subject: [PATCH 08/27] chore: Added a number of physical planning join benchmarks (#13085) * chore: Added a number of physical planning join benchmarks * Ran cargo fmt --- datafusion/core/benches/sql_planner.rs | 69 +++++++++++++++++++++++++- 1 file changed, 68 insertions(+), 1 deletion(-) diff --git a/datafusion/core/benches/sql_planner.rs b/datafusion/core/benches/sql_planner.rs index 09f05c70fec6..64d2760e9d97 100644 --- a/datafusion/core/benches/sql_planner.rs +++ b/datafusion/core/benches/sql_planner.rs @@ -60,7 +60,9 @@ fn create_schema(column_prefix: &str, num_columns: usize) -> Schema { fn create_table_provider(column_prefix: &str, num_columns: usize) -> Arc { let schema = Arc::new(create_schema(column_prefix, num_columns)); - MemTable::try_new(schema, vec![]).map(Arc::new).unwrap() + MemTable::try_new(schema, vec![vec![]]) + .map(Arc::new) + .unwrap() } fn create_context() -> SessionContext { @@ -158,6 +160,71 @@ fn criterion_benchmark(c: &mut Criterion) { }); }); + // Benchmark for Physical Planning Joins + c.bench_function("physical_join_consider_sort", |b| { + b.iter(|| { + physical_plan( + &ctx, + "SELECT t1.a7, t2.b8 \ + FROM t1, t2 WHERE a7 = b7 \ + ORDER BY a7", + ); + }); + }); + + c.bench_function("physical_theta_join_consider_sort", |b| { + b.iter(|| { + physical_plan( + &ctx, + "SELECT t1.a7, t2.b8 \ + FROM t1, t2 WHERE a7 < b7 \ + ORDER BY a7", + ); + }); + }); + + c.bench_function("physical_many_self_joins", |b| { + b.iter(|| { + physical_plan( + &ctx, + "SELECT ta.a9, tb.a10, tc.a11, td.a12, te.a13, tf.a14 \ + FROM t1 AS ta, t1 AS tb, t1 AS tc, t1 AS td, t1 AS te, t1 AS tf \ + WHERE ta.a9 = tb.a10 AND tb.a10 = tc.a11 AND tc.a11 = td.a12 AND \ + td.a12 = te.a13 AND te.a13 = tf.a14", + ); + }); + }); + + c.bench_function("physical_unnest_to_join", |b| { + b.iter(|| { + physical_plan( + &ctx, + "SELECT t1.a7 \ + FROM t1 WHERE a7 = (SELECT b8 FROM t2)", + ); + }); + }); + + c.bench_function("physical_intersection", |b| { + b.iter(|| { + physical_plan( + &ctx, + "SELECT t1.a7 FROM t1 \ + INTERSECT SELECT t2.b8 FROM t2", + ); + }); + }); + // these two queries should be equivalent + c.bench_function("physical_join_distinct", |b| { + b.iter(|| { + logical_plan( + &ctx, + "SELECT DISTINCT t1.a7 \ + FROM t1, t2 WHERE t1.a7 = t2.b8", + ); + }); + }); + // --- TPC-H --- let tpch_ctx = register_defs(SessionContext::new(), tpch_schemas()); From 9e636d8fe920340409e527da36cff3d2c25aef5f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Burak=20=C5=9Een?= Date: Thu, 24 Oct 2024 23:28:13 +0300 Subject: [PATCH 09/27] [docs]: migrate lead/lag window function docs to new docs (#13095) * added lead-lag docs * deleted old --- datafusion/functions-window/src/lead_lag.rs | 58 ++++++++++++++++++- .../source/user-guide/sql/window_functions.md | 30 ---------- .../user-guide/sql/window_functions_new.md | 33 +++++++++++ 3 files changed, 88 insertions(+), 33 deletions(-) diff --git a/datafusion/functions-window/src/lead_lag.rs b/datafusion/functions-window/src/lead_lag.rs index f81521099751..bbe50cbbdc8a 100644 --- a/datafusion/functions-window/src/lead_lag.rs +++ b/datafusion/functions-window/src/lead_lag.rs @@ -22,9 +22,10 @@ use datafusion_common::arrow::array::ArrayRef; use datafusion_common::arrow::datatypes::DataType; use datafusion_common::arrow::datatypes::Field; use datafusion_common::{arrow_datafusion_err, DataFusionError, Result, ScalarValue}; +use datafusion_expr::window_doc_sections::DOC_SECTION_ANALYTICAL; use datafusion_expr::{ - Literal, PartitionEvaluator, ReversedUDWF, Signature, TypeSignature, Volatility, - WindowUDFImpl, + Documentation, Literal, PartitionEvaluator, ReversedUDWF, Signature, TypeSignature, + Volatility, WindowUDFImpl, }; use datafusion_functions_window_common::expr::ExpressionArgs; use datafusion_functions_window_common::field::WindowUDFFieldArgs; @@ -34,7 +35,7 @@ use std::any::Any; use std::cmp::min; use std::collections::VecDeque; use std::ops::{Neg, Range}; -use std::sync::Arc; +use std::sync::{Arc, OnceLock}; get_or_init_udwf!( Lag, @@ -147,6 +148,50 @@ impl WindowShift { } } +static LAG_DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_lag_doc() -> &'static Documentation { + LAG_DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_ANALYTICAL) + .with_description( + "Returns value evaluated at the row that is offset rows before the \ + current row within the partition; if there is no such row, instead return default \ + (which must be of the same type as value).", + ) + .with_syntax_example("lag(expression, offset, default)") + .with_argument("expression", "Expression to operate on") + .with_argument("offset", "Integer. Specifies how many rows back \ + the value of expression should be retrieved. Defaults to 1.") + .with_argument("default", "The default value if the offset is \ + not within the partition. Must be of the same type as expression.") + .build() + .unwrap() + }) +} + +static LEAD_DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_lead_doc() -> &'static Documentation { + LEAD_DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_ANALYTICAL) + .with_description( + "Returns value evaluated at the row that is offset rows after the \ + current row within the partition; if there is no such row, instead return default \ + (which must be of the same type as value).", + ) + .with_syntax_example("lead(expression, offset, default)") + .with_argument("expression", "Expression to operate on") + .with_argument("offset", "Integer. Specifies how many rows \ + forward the value of expression should be retrieved. Defaults to 1.") + .with_argument("default", "The default value if the offset is \ + not within the partition. Must be of the same type as expression.") + .build() + .unwrap() + }) +} + impl WindowUDFImpl for WindowShift { fn as_any(&self) -> &dyn Any { self @@ -212,6 +257,13 @@ impl WindowUDFImpl for WindowShift { WindowShiftKind::Lead => ReversedUDWF::Reversed(lead_udwf()), } } + + fn documentation(&self) -> Option<&Documentation> { + match self.kind { + WindowShiftKind::Lag => Some(get_lag_doc()), + WindowShiftKind::Lead => Some(get_lead_doc()), + } + } } /// When `lead`/`lag` is evaluated on a `NULL` expression we attempt to diff --git a/docs/source/user-guide/sql/window_functions.md b/docs/source/user-guide/sql/window_functions.md index 6c0de711bc0c..0799859e4371 100644 --- a/docs/source/user-guide/sql/window_functions.md +++ b/docs/source/user-guide/sql/window_functions.md @@ -184,8 +184,6 @@ ntile(expression) - [cume_dist](#cume_dist) - [percent_rank](#percent_rank) -- [lag](#lag) -- [lead](#lead) - [first_value](#first_value) - [last_value](#last_value) - [nth_value](#nth_value) @@ -206,34 +204,6 @@ Relative rank of the current row: (rank - 1) / (total rows - 1). percent_rank() ``` -### `lag` - -Returns value evaluated at the row that is offset rows before the current row within the partition; if there is no such row, instead return default (which must be of the same type as value). Both offset and default are evaluated with respect to the current row. If omitted, offset defaults to 1 and default to null. - -```sql -lag(expression, offset, default) -``` - -#### Arguments - -- **expression**: Expression to operate on -- **offset**: Integer. Specifies how many rows back the value of _expression_ should be retrieved. Defaults to 1. -- **default**: The default value if the offset is not within the partition. Must be of the same type as _expression_. - -### `lead` - -Returns value evaluated at the row that is offset rows after the current row within the partition; if there is no such row, instead return default (which must be of the same type as value). Both offset and default are evaluated with respect to the current row. If omitted, offset defaults to 1 and default to null. - -```sql -lead(expression, offset, default) -``` - -#### Arguments - -- **expression**: Expression to operate on -- **offset**: Integer. Specifies how many rows forward the value of _expression_ should be retrieved. Defaults to 1. -- **default**: The default value if the offset is not within the partition. Must be of the same type as _expression_. - ### `first_value` Returns value evaluated at the row that is the first row of the window frame. diff --git a/docs/source/user-guide/sql/window_functions_new.md b/docs/source/user-guide/sql/window_functions_new.md index 89ce2284a70c..267060abfdcc 100644 --- a/docs/source/user-guide/sql/window_functions_new.md +++ b/docs/source/user-guide/sql/window_functions_new.md @@ -202,3 +202,36 @@ Number of the current row within its partition, counting from 1. ``` row_number() ``` + +## Analytical Functions + +- [lag](#lag) +- [lead](#lead) + +### `lag` + +Returns value evaluated at the row that is offset rows before the current row within the partition; if there is no such row, instead return default (which must be of the same type as value). + +``` +lag(expression, offset, default) +``` + +#### Arguments + +- **expression**: Expression to operate on +- **offset**: Integer. Specifies how many rows back the value of expression should be retrieved. Defaults to 1. +- **default**: The default value if the offset is not within the partition. Must be of the same type as expression. + +### `lead` + +Returns value evaluated at the row that is offset rows after the current row within the partition; if there is no such row, instead return default (which must be of the same type as value). + +``` +lead(expression, offset, default) +``` + +#### Arguments + +- **expression**: Expression to operate on +- **offset**: Integer. Specifies how many rows forward the value of expression should be retrieved. Defaults to 1. +- **default**: The default value if the offset is not within the partition. Must be of the same type as expression. From 232293367d6f3ff10e291597e1dc45bcce7de7d7 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Thu, 24 Oct 2024 13:29:33 -0700 Subject: [PATCH 10/27] minor: Add deprecated policy to the contributor guide contents and fix the link from main README (#13100) --- README.md | 2 +- docs/source/index.rst | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index bbbdf7133518..f89935d597c2 100644 --- a/README.md +++ b/README.md @@ -138,4 +138,4 @@ We enforce this policy using a [MSRV CI Check](https://github.com/search?q=repo% ## DataFusion API evolution policy Public methods in Apache DataFusion are subject to evolve as part of the API lifecycle. -Deprecated methods will be phased out in accordance with the [policy](docs/source/library-user-guide/api-health.md), ensuring the API is stable and healthy. +Deprecated methods will be phased out in accordance with the [policy](https://datafusion.apache.org/library-user-guide/api-health.html), ensuring the API is stable and healthy. diff --git a/docs/source/index.rst b/docs/source/index.rst index 27dd58cf50f4..9008950d3dd6 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -130,6 +130,7 @@ To get started, see library-user-guide/extending-operators library-user-guide/profiling library-user-guide/query-optimizer + library-user-guide/api-health .. _toc.contributor-guide: .. toctree:: From 6a3c0b0bce67553a4431b941d13fc995f310bee8 Mon Sep 17 00:00:00 2001 From: Filippo Rossi Date: Fri, 25 Oct 2024 08:28:41 +0200 Subject: [PATCH 11/27] feat: improve type inference for WindowFrame (#13059) * feat: improve type inference for WindowFrame Closes #11432 * Support Interval for groups and rows * Remove case for SingleQuotedString --- datafusion/expr/src/window_frame.rs | 191 ++++++++++++++---- .../optimizer/src/analyzer/type_coercion.rs | 23 ++- datafusion/sql/tests/cases/plan_to_sql.rs | 2 +- datafusion/sqllogictest/test_files/window.slt | 32 ++- .../substrait/src/logical_plan/producer.rs | 104 ++-------- 5 files changed, 217 insertions(+), 135 deletions(-) diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs index b2e8268aa332..349968c3fa2f 100644 --- a/datafusion/expr/src/window_frame.rs +++ b/datafusion/expr/src/window_frame.rs @@ -23,11 +23,11 @@ //! - An ending frame boundary, //! - An EXCLUDE clause. +use crate::{expr::Sort, lit}; +use arrow::datatypes::DataType; use std::fmt::{self, Formatter}; use std::hash::Hash; -use crate::{expr::Sort, lit}; - use datafusion_common::{plan_err, sql_err, DataFusionError, Result, ScalarValue}; use sqlparser::ast; use sqlparser::parser::ParserError::ParserError; @@ -119,9 +119,9 @@ impl TryFrom for WindowFrame { type Error = DataFusionError; fn try_from(value: ast::WindowFrame) -> Result { - let start_bound = value.start_bound.try_into()?; + let start_bound = WindowFrameBound::try_parse(value.start_bound, &value.units)?; let end_bound = match value.end_bound { - Some(value) => value.try_into()?, + Some(bound) => WindowFrameBound::try_parse(bound, &value.units)?, None => WindowFrameBound::CurrentRow, }; @@ -138,6 +138,7 @@ impl TryFrom for WindowFrame { )? } }; + let units = value.units.into(); Ok(Self::new_bounds(units, start_bound, end_bound)) } @@ -334,17 +335,18 @@ impl WindowFrameBound { } } -impl TryFrom for WindowFrameBound { - type Error = DataFusionError; - - fn try_from(value: ast::WindowFrameBound) -> Result { +impl WindowFrameBound { + fn try_parse( + value: ast::WindowFrameBound, + units: &ast::WindowFrameUnits, + ) -> Result { Ok(match value { ast::WindowFrameBound::Preceding(Some(v)) => { - Self::Preceding(convert_frame_bound_to_scalar_value(*v)?) + Self::Preceding(convert_frame_bound_to_scalar_value(*v, units)?) } ast::WindowFrameBound::Preceding(None) => Self::Preceding(ScalarValue::Null), ast::WindowFrameBound::Following(Some(v)) => { - Self::Following(convert_frame_bound_to_scalar_value(*v)?) + Self::Following(convert_frame_bound_to_scalar_value(*v, units)?) } ast::WindowFrameBound::Following(None) => Self::Following(ScalarValue::Null), ast::WindowFrameBound::CurrentRow => Self::CurrentRow, @@ -352,33 +354,65 @@ impl TryFrom for WindowFrameBound { } } -pub fn convert_frame_bound_to_scalar_value(v: ast::Expr) -> Result { - Ok(ScalarValue::Utf8(Some(match v { - ast::Expr::Value(ast::Value::Number(value, false)) - | ast::Expr::Value(ast::Value::SingleQuotedString(value)) => value, - ast::Expr::Interval(ast::Interval { - value, - leading_field, - .. - }) => { - let result = match *value { - ast::Expr::Value(ast::Value::SingleQuotedString(item)) => item, - e => { - return sql_err!(ParserError(format!( - "INTERVAL expression cannot be {e:?}" - ))); +fn convert_frame_bound_to_scalar_value( + v: ast::Expr, + units: &ast::WindowFrameUnits, +) -> Result { + match units { + // For ROWS and GROUPS we are sure that the ScalarValue must be a non-negative integer ... + ast::WindowFrameUnits::Rows | ast::WindowFrameUnits::Groups => match v { + ast::Expr::Value(ast::Value::Number(value, false)) => { + Ok(ScalarValue::try_from_string(value, &DataType::UInt64)?) + }, + ast::Expr::Interval(ast::Interval { + value, + leading_field: None, + leading_precision: None, + last_field: None, + fractional_seconds_precision: None, + }) => { + let value = match *value { + ast::Expr::Value(ast::Value::SingleQuotedString(item)) => item, + e => { + return sql_err!(ParserError(format!( + "INTERVAL expression cannot be {e:?}" + ))); + } + }; + Ok(ScalarValue::try_from_string(value, &DataType::UInt64)?) + } + _ => plan_err!( + "Invalid window frame: frame offsets for ROWS / GROUPS must be non negative integers" + ), + }, + // ... instead for RANGE it could be anything depending on the type of the ORDER BY clause, + // so we use a ScalarValue::Utf8. + ast::WindowFrameUnits::Range => Ok(ScalarValue::Utf8(Some(match v { + ast::Expr::Value(ast::Value::Number(value, false)) => value, + ast::Expr::Interval(ast::Interval { + value, + leading_field, + .. + }) => { + let result = match *value { + ast::Expr::Value(ast::Value::SingleQuotedString(item)) => item, + e => { + return sql_err!(ParserError(format!( + "INTERVAL expression cannot be {e:?}" + ))); + } + }; + if let Some(leading_field) = leading_field { + format!("{result} {leading_field}") + } else { + result } - }; - if let Some(leading_field) = leading_field { - format!("{result} {leading_field}") - } else { - result } - } - _ => plan_err!( - "Invalid window frame: frame offsets must be non negative integers" - )?, - }))) + _ => plan_err!( + "Invalid window frame: frame offsets for RANGE must be either a numeric value, a string value or an interval" + )?, + }))), + } } impl fmt::Display for WindowFrameBound { @@ -479,8 +513,91 @@ mod tests { ast::Expr::Value(ast::Value::Number("1".to_string(), false)), )))), }; - let result = WindowFrame::try_from(window_frame); - assert!(result.is_ok()); + + let window_frame = WindowFrame::try_from(window_frame)?; + assert_eq!(window_frame.units, WindowFrameUnits::Rows); + assert_eq!( + window_frame.start_bound, + WindowFrameBound::Preceding(ScalarValue::UInt64(Some(2))) + ); + assert_eq!( + window_frame.end_bound, + WindowFrameBound::Preceding(ScalarValue::UInt64(Some(1))) + ); + + Ok(()) + } + + macro_rules! test_bound { + ($unit:ident, $value:expr, $expected:expr) => { + let preceding = WindowFrameBound::try_parse( + ast::WindowFrameBound::Preceding($value), + &ast::WindowFrameUnits::$unit, + )?; + assert_eq!(preceding, WindowFrameBound::Preceding($expected)); + let following = WindowFrameBound::try_parse( + ast::WindowFrameBound::Following($value), + &ast::WindowFrameUnits::$unit, + )?; + assert_eq!(following, WindowFrameBound::Following($expected)); + }; + } + + macro_rules! test_bound_err { + ($unit:ident, $value:expr, $expected:expr) => { + let err = WindowFrameBound::try_parse( + ast::WindowFrameBound::Preceding($value), + &ast::WindowFrameUnits::$unit, + ) + .unwrap_err(); + assert_eq!(err.strip_backtrace(), $expected); + let err = WindowFrameBound::try_parse( + ast::WindowFrameBound::Following($value), + &ast::WindowFrameUnits::$unit, + ) + .unwrap_err(); + assert_eq!(err.strip_backtrace(), $expected); + }; + } + + #[test] + fn test_window_frame_bound_creation() -> Result<()> { + // Unbounded + test_bound!(Rows, None, ScalarValue::Null); + test_bound!(Groups, None, ScalarValue::Null); + test_bound!(Range, None, ScalarValue::Null); + + // Number + let number = Some(Box::new(ast::Expr::Value(ast::Value::Number( + "42".to_string(), + false, + )))); + test_bound!(Rows, number.clone(), ScalarValue::UInt64(Some(42))); + test_bound!(Groups, number.clone(), ScalarValue::UInt64(Some(42))); + test_bound!( + Range, + number.clone(), + ScalarValue::Utf8(Some("42".to_string())) + ); + + // Interval + let number = Some(Box::new(ast::Expr::Interval(ast::Interval { + value: Box::new(ast::Expr::Value(ast::Value::SingleQuotedString( + "1".to_string(), + ))), + leading_field: Some(ast::DateTimeField::Day), + fractional_seconds_precision: None, + last_field: None, + leading_precision: None, + }))); + test_bound_err!(Rows, number.clone(), "Error during planning: Invalid window frame: frame offsets for ROWS / GROUPS must be non negative integers"); + test_bound_err!(Groups, number.clone(), "Error during planning: Invalid window frame: frame offsets for ROWS / GROUPS must be non negative integers"); + test_bound!( + Range, + number.clone(), + ScalarValue::Utf8(Some("1 DAY".to_string())) + ); + Ok(()) } } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 36b72233b5af..33eea1a661c6 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -696,20 +696,20 @@ fn coerce_window_frame( expressions: &[Sort], ) -> Result { let mut window_frame = window_frame; - let current_types = expressions - .iter() - .map(|s| s.expr.get_type(schema)) - .collect::>>()?; let target_type = match window_frame.units { WindowFrameUnits::Range => { - if let Some(col_type) = current_types.first() { + let current_types = expressions + .first() + .map(|s| s.expr.get_type(schema)) + .transpose()?; + if let Some(col_type) = current_types { if col_type.is_numeric() - || is_utf8_or_large_utf8(col_type) + || is_utf8_or_large_utf8(&col_type) || matches!(col_type, DataType::Null) { col_type - } else if is_datetime(col_type) { - &DataType::Interval(IntervalUnit::MonthDayNano) + } else if is_datetime(&col_type) { + DataType::Interval(IntervalUnit::MonthDayNano) } else { return internal_err!( "Cannot run range queries on datatype: {col_type:?}" @@ -719,10 +719,11 @@ fn coerce_window_frame( return internal_err!("ORDER BY column cannot be empty"); } } - WindowFrameUnits::Rows | WindowFrameUnits::Groups => &DataType::UInt64, + WindowFrameUnits::Rows | WindowFrameUnits::Groups => DataType::UInt64, }; - window_frame.start_bound = coerce_frame_bound(target_type, window_frame.start_bound)?; - window_frame.end_bound = coerce_frame_bound(target_type, window_frame.end_bound)?; + window_frame.start_bound = + coerce_frame_bound(&target_type, window_frame.start_bound)?; + window_frame.end_bound = coerce_frame_bound(&target_type, window_frame.end_bound)?; Ok(window_frame) } diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index 9ed084eec249..8e25c1c5b1cd 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -1061,7 +1061,7 @@ fn test_aggregation_to_sql() { FROM person GROUP BY id, first_name;"#, r#"SELECT person.id, person.first_name, -sum(person.id) AS total_sum, sum(person.id) OVER (PARTITION BY person.first_name ROWS BETWEEN '5' PRECEDING AND '2' FOLLOWING) AS moving_sum, +sum(person.id) AS total_sum, sum(person.id) OVER (PARTITION BY person.first_name ROWS BETWEEN 5 PRECEDING AND 2 FOLLOWING) AS moving_sum, max(sum(person.id)) OVER (PARTITION BY person.first_name ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS max_total, rank() OVER (PARTITION BY (grouping(person.id) + grouping(person.age)), CASE WHEN (grouping(person.age) = 0) THEN person.id END ORDER BY sum(person.id) DESC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS rank_within_parent_1, rank() OVER (PARTITION BY (grouping(person.age) + grouping(person.id)), CASE WHEN (CAST(grouping(person.age) AS BIGINT) = 0) THEN person.id END ORDER BY sum(person.id) DESC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS rank_within_parent_2 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 95d850795772..4a2d9e1d6864 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2208,7 +2208,7 @@ physical_plan 01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] 02)--SortExec: TopK(fetch=5), expr=[c9@2 ASC NULLS LAST], preserve_partitioning=[false] 03)----ProjectionExec: expr=[sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum1, sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING@4 as sum2, c9@1 as c9] -04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING: Ok(Field { name: "sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(3)), is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING: Ok(Field { name: "sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(3)), is_causal: true }], mode=[Sorted] 05)--------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 07)------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[false] @@ -2378,17 +2378,41 @@ SELECT c9, rn1 FROM (SELECT c9, # invalid window frame. null as preceding -statement error DataFusion error: Error during planning: Invalid window frame: frame offsets must be non negative integers +statement error DataFusion error: Error during planning: Invalid window frame: frame offsets for ROWS / GROUPS must be non negative integers select row_number() over (rows between null preceding and current row) from (select 1 a) x # invalid window frame. null as preceding -statement error DataFusion error: Error during planning: Invalid window frame: frame offsets must be non negative integers +statement error DataFusion error: Error during planning: Invalid window frame: frame offsets for ROWS / GROUPS must be non negative integers select row_number() over (rows between null preceding and current row) from (select 1 a) x # invalid window frame. negative as following -statement error DataFusion error: Error during planning: Invalid window frame: frame offsets must be non negative integers +statement error DataFusion error: Error during planning: Invalid window frame: frame offsets for ROWS / GROUPS must be non negative integers select row_number() over (rows between current row and -1 following) from (select 1 a) x +# invalid window frame. null as preceding +statement error DataFusion error: Error during planning: Invalid window frame: frame offsets for ROWS / GROUPS must be non negative integers +select row_number() over (order by a groups between null preceding and current row) from (select 1 a) x + +# invalid window frame. null as preceding +statement error DataFusion error: Error during planning: Invalid window frame: frame offsets for ROWS / GROUPS must be non negative integers +select row_number() over (order by a groups between null preceding and current row) from (select 1 a) x + +# invalid window frame. negative as following +statement error DataFusion error: Error during planning: Invalid window frame: frame offsets for ROWS / GROUPS must be non negative integers +select row_number() over (order by a groups between current row and -1 following) from (select 1 a) x + +# interval for rows +query I +select row_number() over (rows between '1' preceding and current row) from (select 1 a) x +---- +1 + +# interval for groups +query I +select row_number() over (order by a groups between '1' preceding and current row) from (select 1 a) x +---- +1 + # This test shows that ordering satisfy considers ordering equivalences, # and can simplify (reduce expression size) multi expression requirements during normalization # For the example below, requirement rn1 ASC, c9 DESC should be simplified to the rn1 ASC. diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 4105dc1876db..4855af683b7d 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -1718,98 +1718,38 @@ fn make_substrait_like_expr( } } +fn to_substrait_bound_offset(value: &ScalarValue) -> Option { + match value { + ScalarValue::UInt8(Some(v)) => Some(*v as i64), + ScalarValue::UInt16(Some(v)) => Some(*v as i64), + ScalarValue::UInt32(Some(v)) => Some(*v as i64), + ScalarValue::UInt64(Some(v)) => Some(*v as i64), + ScalarValue::Int8(Some(v)) => Some(*v as i64), + ScalarValue::Int16(Some(v)) => Some(*v as i64), + ScalarValue::Int32(Some(v)) => Some(*v as i64), + ScalarValue::Int64(Some(v)) => Some(*v), + _ => None, + } +} + fn to_substrait_bound(bound: &WindowFrameBound) -> Bound { match bound { WindowFrameBound::CurrentRow => Bound { kind: Some(BoundKind::CurrentRow(SubstraitBound::CurrentRow {})), }, - WindowFrameBound::Preceding(s) => match s { - ScalarValue::UInt8(Some(v)) => Bound { - kind: Some(BoundKind::Preceding(SubstraitBound::Preceding { - offset: *v as i64, - })), + WindowFrameBound::Preceding(s) => match to_substrait_bound_offset(s) { + Some(offset) => Bound { + kind: Some(BoundKind::Preceding(SubstraitBound::Preceding { offset })), }, - ScalarValue::UInt16(Some(v)) => Bound { - kind: Some(BoundKind::Preceding(SubstraitBound::Preceding { - offset: *v as i64, - })), - }, - ScalarValue::UInt32(Some(v)) => Bound { - kind: Some(BoundKind::Preceding(SubstraitBound::Preceding { - offset: *v as i64, - })), - }, - ScalarValue::UInt64(Some(v)) => Bound { - kind: Some(BoundKind::Preceding(SubstraitBound::Preceding { - offset: *v as i64, - })), - }, - ScalarValue::Int8(Some(v)) => Bound { - kind: Some(BoundKind::Preceding(SubstraitBound::Preceding { - offset: *v as i64, - })), - }, - ScalarValue::Int16(Some(v)) => Bound { - kind: Some(BoundKind::Preceding(SubstraitBound::Preceding { - offset: *v as i64, - })), - }, - ScalarValue::Int32(Some(v)) => Bound { - kind: Some(BoundKind::Preceding(SubstraitBound::Preceding { - offset: *v as i64, - })), - }, - ScalarValue::Int64(Some(v)) => Bound { - kind: Some(BoundKind::Preceding(SubstraitBound::Preceding { - offset: *v, - })), - }, - _ => Bound { + None => Bound { kind: Some(BoundKind::Unbounded(SubstraitBound::Unbounded {})), }, }, - WindowFrameBound::Following(s) => match s { - ScalarValue::UInt8(Some(v)) => Bound { - kind: Some(BoundKind::Following(SubstraitBound::Following { - offset: *v as i64, - })), - }, - ScalarValue::UInt16(Some(v)) => Bound { - kind: Some(BoundKind::Following(SubstraitBound::Following { - offset: *v as i64, - })), - }, - ScalarValue::UInt32(Some(v)) => Bound { - kind: Some(BoundKind::Following(SubstraitBound::Following { - offset: *v as i64, - })), - }, - ScalarValue::UInt64(Some(v)) => Bound { - kind: Some(BoundKind::Following(SubstraitBound::Following { - offset: *v as i64, - })), - }, - ScalarValue::Int8(Some(v)) => Bound { - kind: Some(BoundKind::Following(SubstraitBound::Following { - offset: *v as i64, - })), - }, - ScalarValue::Int16(Some(v)) => Bound { - kind: Some(BoundKind::Following(SubstraitBound::Following { - offset: *v as i64, - })), - }, - ScalarValue::Int32(Some(v)) => Bound { - kind: Some(BoundKind::Following(SubstraitBound::Following { - offset: *v as i64, - })), - }, - ScalarValue::Int64(Some(v)) => Bound { - kind: Some(BoundKind::Following(SubstraitBound::Following { - offset: *v, - })), + WindowFrameBound::Following(s) => match to_substrait_bound_offset(s) { + Some(offset) => Bound { + kind: Some(BoundKind::Following(SubstraitBound::Following { offset })), }, - _ => Bound { + None => Bound { kind: Some(BoundKind::Unbounded(SubstraitBound::Unbounded {})), }, }, From 13a422579d3d0d68c90ee31fdeb5e9bb4bd2df7f Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Fri, 25 Oct 2024 21:02:54 +0800 Subject: [PATCH 12/27] Introduce `binary_as_string` parquet option, upgrade to arrow/parquet `53.2.0` (#12816) * Update to arrow-rs 53.2.0 * introduce binary_as_string parquet option * Fix test --------- Co-authored-by: Andrew Lamb --- Cargo.toml | 18 +- benchmarks/src/clickbench.rs | 15 +- datafusion-cli/Cargo.lock | 122 +++++------ datafusion/common/src/config.rs | 8 + .../common/src/file_options/parquet_writer.rs | 3 + .../core/src/datasource/file_format/mod.rs | 112 ++++++++-- .../src/datasource/file_format/parquet.rs | 37 +++- .../physical_plan/parquet/opener.rs | 23 +- .../proto/datafusion_common.proto | 1 + datafusion/proto-common/src/from_proto/mod.rs | 11 +- .../proto-common/src/generated/pbjson.rs | 75 ++++++- .../proto-common/src/generated/prost.rs | 190 +++++------------ datafusion/proto-common/src/to_proto/mod.rs | 1 + .../src/generated/datafusion_proto_common.rs | 3 + .../proto/src/logical_plan/file_formats.rs | 2 + .../test_files/information_schema.slt | 2 + .../sqllogictest/test_files/parquet.slt | 201 ++++++++++++++++++ docs/source/user-guide/configs.md | 1 + 18 files changed, 581 insertions(+), 244 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 63bfb7fce413..e1e3aca77153 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -70,22 +70,22 @@ version = "42.1.0" ahash = { version = "0.8", default-features = false, features = [ "runtime-rng", ] } -arrow = { version = "53.1.0", features = [ +arrow = { version = "53.2.0", features = [ "prettyprint", ] } -arrow-array = { version = "53.1.0", default-features = false, features = [ +arrow-array = { version = "53.2.0", default-features = false, features = [ "chrono-tz", ] } -arrow-buffer = { version = "53.1.0", default-features = false } -arrow-flight = { version = "53.1.0", features = [ +arrow-buffer = { version = "53.2.0", default-features = false } +arrow-flight = { version = "53.2.0", features = [ "flight-sql-experimental", ] } -arrow-ipc = { version = "53.1.0", default-features = false, features = [ +arrow-ipc = { version = "53.2.0", default-features = false, features = [ "lz4", ] } -arrow-ord = { version = "53.1.0", default-features = false } -arrow-schema = { version = "53.1.0", default-features = false } -arrow-string = { version = "53.1.0", default-features = false } +arrow-ord = { version = "53.2.0", default-features = false } +arrow-schema = { version = "53.2.0", default-features = false } +arrow-string = { version = "53.2.0", default-features = false } async-trait = "0.1.73" bigdecimal = "=0.4.1" bytes = "1.4" @@ -126,7 +126,7 @@ log = "^0.4" num_cpus = "1.13.0" object_store = { version = "0.11.0", default-features = false } parking_lot = "0.12" -parquet = { version = "53.1.0", default-features = false, features = [ +parquet = { version = "53.2.0", default-features = false, features = [ "arrow", "async", "object_store", diff --git a/benchmarks/src/clickbench.rs b/benchmarks/src/clickbench.rs index 6ebefa985b51..3564ae82585a 100644 --- a/benchmarks/src/clickbench.rs +++ b/benchmarks/src/clickbench.rs @@ -115,12 +115,15 @@ impl RunOpt { None => queries.min_query_id()..=queries.max_query_id(), }; + // configure parquet options let mut config = self.common.config(); - config - .options_mut() - .execution - .parquet - .schema_force_view_types = self.common.force_view_types; + { + let parquet_options = &mut config.options_mut().execution.parquet; + parquet_options.schema_force_view_types = self.common.force_view_types; + // The hits_partitioned dataset specifies string columns + // as binary due to how it was written. Force it to strings + parquet_options.binary_as_string = true; + } let ctx = SessionContext::new_with_config(config); self.register_hits(&ctx).await?; @@ -148,7 +151,7 @@ impl RunOpt { Ok(()) } - /// Registrs the `hits.parquet` as a table named `hits` + /// Registers the `hits.parquet` as a table named `hits` async fn register_hits(&self, ctx: &SessionContext) -> Result<()> { let options = Default::default(); let path = self.path.as_os_str().to_str().unwrap(); diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 24649832b27e..ca67e3e4f531 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -84,9 +84,9 @@ dependencies = [ [[package]] name = "anstream" -version = "0.6.15" +version = "0.6.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64e15c1ab1f89faffbf04a634d5e1962e9074f2741eef6d97f3c4e322426d526" +checksum = "5f581a3637024bb8f62027f3ab6151f502090388c1dad05b01c70fb733b33c20" dependencies = [ "anstyle", "anstyle-parse", @@ -123,12 +123,12 @@ dependencies = [ [[package]] name = "anstyle-wincon" -version = "3.0.4" +version = "3.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bf74e1b6e971609db8ca7a9ce79fd5768ab6ae46441c572e46cf596f59e57f8" +checksum = "abbf7eaf69f3b46121caf74645dd5d3078b4b205a2513930da0033156682cd28" dependencies = [ "anstyle", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -173,9 +173,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9ba0d7248932f4e2a12fb37f0a2e3ec82b3bdedbac2a1dce186e036843b8f8c" +checksum = "4caf25cdc4a985f91df42ed9e9308e1adbcd341a31a72605c697033fcef163e3" dependencies = [ "arrow-arith", "arrow-array", @@ -194,9 +194,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d60afcdc004841a5c8d8da4f4fa22d64eb19c0c01ef4bcedd77f175a7cf6e38f" +checksum = "91f2dfd1a7ec0aca967dfaa616096aec49779adc8eccec005e2f5e4111b1192a" dependencies = [ "arrow-array", "arrow-buffer", @@ -209,9 +209,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f16835e8599dbbb1659fd869d865254c4cf32c6c2bb60b6942ac9fc36bfa5da" +checksum = "d39387ca628be747394890a6e47f138ceac1aa912eab64f02519fed24b637af8" dependencies = [ "ahash", "arrow-buffer", @@ -226,9 +226,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a1f34f0faae77da6b142db61deba2cb6d60167592b178be317b341440acba80" +checksum = "9e51e05228852ffe3eb391ce7178a0f97d2cf80cc6ef91d3c4a6b3cb688049ec" dependencies = [ "bytes", "half", @@ -237,9 +237,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "450e4abb5775bca0740bec0bcf1b1a5ae07eff43bd625661c4436d8e8e4540c4" +checksum = "d09aea56ec9fa267f3f3f6cdab67d8a9974cbba90b3aa38c8fe9d0bb071bd8c1" dependencies = [ "arrow-array", "arrow-buffer", @@ -258,9 +258,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3a4e4d63830a341713e35d9a42452fbc6241d5f42fa5cf6a4681b8ad91370c4" +checksum = "c07b5232be87d115fde73e32f2ca7f1b353bff1b44ac422d3c6fc6ae38f11f0d" dependencies = [ "arrow-array", "arrow-buffer", @@ -277,9 +277,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b1e618bbf714c7a9e8d97203c806734f012ff71ae3adc8ad1b075689f540634" +checksum = "b98ae0af50890b494cebd7d6b04b35e896205c1d1df7b29a6272c5d0d0249ef5" dependencies = [ "arrow-buffer", "arrow-schema", @@ -289,9 +289,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f98e983549259a2b97049af7edfb8f28b8911682040e99a94e4ceb1196bd65c2" +checksum = "0ed91bdeaff5a1c00d28d8f73466bcb64d32bbd7093b5a30156b4b9f4dba3eee" dependencies = [ "arrow-array", "arrow-buffer", @@ -304,9 +304,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b198b9c6fcf086501730efbbcb483317b39330a116125af7bb06467d04b352a3" +checksum = "0471f51260a5309307e5d409c9dc70aede1cd9cf1d4ff0f0a1e8e1a2dd0e0d3c" dependencies = [ "arrow-array", "arrow-buffer", @@ -324,9 +324,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2427f37b4459a4b9e533045abe87a5183a5e0995a3fc2c2fd45027ae2cc4ef3f" +checksum = "2883d7035e0b600fb4c30ce1e50e66e53d8656aa729f2bfa4b51d359cf3ded52" dependencies = [ "arrow-array", "arrow-buffer", @@ -339,9 +339,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15959657d92e2261a7a323517640af87f5afd9fd8a6492e424ebee2203c567f6" +checksum = "552907e8e587a6fde4f8843fd7a27a576a260f65dab6c065741ea79f633fc5be" dependencies = [ "ahash", "arrow-array", @@ -353,15 +353,15 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbf0388a18fd7f7f3fe3de01852d30f54ed5182f9004db700fbe3ba843ed2794" +checksum = "539ada65246b949bd99ffa0881a9a15a4a529448af1a07a9838dd78617dafab1" [[package]] name = "arrow-select" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b83e5723d307a38bf00ecd2972cd078d1339c7fd3eb044f609958a9a24463f3a" +checksum = "6259e566b752da6dceab91766ed8b2e67bf6270eb9ad8a6e07a33c1bede2b125" dependencies = [ "ahash", "arrow-array", @@ -373,9 +373,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ab3db7c09dd826e74079661d84ed01ed06547cf75d52c2818ef776d0d852305" +checksum = "f3179ccbd18ebf04277a095ba7321b93fd1f774f18816bd5f6b3ce2f594edb6c" dependencies = [ "arrow-array", "arrow-buffer", @@ -663,9 +663,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.7.2" +version = "1.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a065c0fe6fdbdf9f11817eb68582b2ab4aff9e9c39e986ae48f7ec576c6322db" +checksum = "be28bd063fa91fd871d131fc8b68d7cd4c5fa0869bea68daca50dcb1cbd76be2" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -707,9 +707,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.2.7" +version = "1.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "147100a7bea70fa20ef224a6bad700358305f5dc0f84649c53769761395b355b" +checksum = "07c9cdc179e6afbf5d391ab08c85eac817b51c87e1892a5edb5f7bbdc64314b4" dependencies = [ "base64-simd", "bytes", @@ -836,9 +836,9 @@ dependencies = [ [[package]] name = "brotli" -version = "6.0.0" +version = "7.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74f7971dbd9326d58187408ab83117d8ac1bb9c17b085fdacd1cf2f598719b6b" +checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -880,9 +880,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.7.2" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "428d9aa8fbc0670b7b8d6030a7fadd0f86151cae55e4dbbece15f3780a3dfaf3" +checksum = "9ac0150caa2ae65ca5bd83f25c7de183dea78d4d366469f148435e2acfbad0da" [[package]] name = "bytes-utils" @@ -2704,9 +2704,9 @@ dependencies = [ [[package]] name = "parquet" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "310c46a70a3ba90d98fec39fa2da6d9d731e544191da6fb56c9d199484d0dd3e" +checksum = "dea02606ba6f5e856561d8d507dba8bac060aefca2a6c0f1aa1d361fed91ff3e" dependencies = [ "ahash", "arrow-array", @@ -2809,9 +2809,9 @@ dependencies = [ [[package]] name = "pin-project-lite" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bda66fc9667c18cb2758a2ac84d1167245054bcf85d5d1aaa6923f45801bdd02" +checksum = "915a1e146535de9163f3987b8944ed8cf49a18bb0056bcebcdcece385cece4ff" [[package]] name = "pin-utils" @@ -2881,9 +2881,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.88" +version = "1.0.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c3a7fc5db1e57d5a779a352c8cdb57b29aa4c40cc69c3a68a7fedc815fbf2f9" +checksum = "f139b0662de085916d1fb67d2b4169d1addddda1919e696f3252b740b629986e" dependencies = [ "unicode-ident", ] @@ -3023,9 +3023,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.11.0" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38200e5ee88914975b69f657f0801b6f6dccafd44fd9326302a4aaeecfacb1d8" +checksum = "b544ef1b4eac5dc2db33ea63606ae9ffcfac26c1416a2806ae0bf5f56b201191" dependencies = [ "aho-corasick", "memchr", @@ -3393,18 +3393,18 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.210" +version = "1.0.213" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8e3592472072e6e22e0a54d5904d9febf8508f65fb8552499a1abc7d1078c3a" +checksum = "3ea7893ff5e2466df8d720bb615088341b295f849602c6956047f8f80f0e9bc1" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.210" +version = "1.0.213" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "243902eda00fad750862fc144cea25caca5e20d615af0a81bee94ca738f1df1f" +checksum = "7e85ad2009c50b58e87caa8cd6dac16bdf511bbfb7af6c33df902396aa480fa5" dependencies = [ "proc-macro2", "quote", @@ -3607,9 +3607,9 @@ checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "syn" -version = "2.0.82" +version = "2.0.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83540f837a8afc019423a8edb95b52a8effe46957ee402287f4292fae35be021" +checksum = "5023162dfcd14ef8f32034d8bcd4cc5ddc61ef7a247c024a33e24e1f24d21b56" dependencies = [ "proc-macro2", "quote", @@ -3646,18 +3646,18 @@ checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76" [[package]] name = "thiserror" -version = "1.0.64" +version = "1.0.65" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d50af8abc119fb8bb6dbabcfa89656f46f84aa0ac7688088608076ad2b459a84" +checksum = "5d11abd9594d9b38965ef50805c5e469ca9cc6f197f883f717e0269a3057b3d5" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.64" +version = "1.0.65" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08904e7672f5eb876eaaf87e0ce17857500934f4981c4a0ab2b4aa98baac7fc3" +checksum = "ae71770322cbd277e69d762a16c444af02aa0575ac0d174f0b9562d3b37f8602" dependencies = [ "proc-macro2", "quote", @@ -3731,9 +3731,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.40.0" +version = "1.41.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2b070231665d27ad9ec9b8df639893f46727666c6767db40317fbe920a5d998" +checksum = "145f3413504347a2be84393cc8a7d2fb4d863b375909ea59f2158261aa258bbb" dependencies = [ "backtrace", "bytes", diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 47ffe0b1c66b..33e5184d2cac 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -390,6 +390,14 @@ config_namespace! { /// and `Binary/BinaryLarge` with `BinaryView`. pub schema_force_view_types: bool, default = false + /// (reading) If true, parquet reader will read columns of + /// `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. + /// + /// Parquet files generated by some legacy writers do not correctly set + /// the UTF8 flag for strings, causing string columns to be loaded as + /// BLOB instead. + pub binary_as_string: bool, default = false + // The following options affect writing to parquet files // and map to parquet::file::properties::WriterProperties diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 5d553d59da4e..dd9d67d6bb47 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -176,6 +176,7 @@ impl ParquetOptions { maximum_buffered_record_batches_per_stream: _, bloom_filter_on_read: _, // reads not used for writer props schema_force_view_types: _, + binary_as_string: _, // not used for writer props } = self; let mut builder = WriterProperties::builder() @@ -442,6 +443,7 @@ mod tests { .maximum_buffered_record_batches_per_stream, bloom_filter_on_read: defaults.bloom_filter_on_read, schema_force_view_types: defaults.schema_force_view_types, + binary_as_string: defaults.binary_as_string, } } @@ -543,6 +545,7 @@ mod tests { .maximum_buffered_record_batches_per_stream, bloom_filter_on_read: global_options_defaults.bloom_filter_on_read, schema_force_view_types: global_options_defaults.schema_force_view_types, + binary_as_string: global_options_defaults.binary_as_string, }, column_specific_options, key_value_metadata, diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index e16986c660ad..a313a7a9bcb1 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -42,7 +42,7 @@ use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::{ExecutionPlan, Statistics}; -use arrow_schema::{DataType, Field, Schema}; +use arrow_schema::{DataType, Field, FieldRef, Schema}; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{internal_err, not_impl_err, GetExt}; use datafusion_expr::Expr; @@ -235,20 +235,26 @@ pub fn file_type_to_format( } } +/// Create a new field with the specified data type, copying the other +/// properties from the input field +fn field_with_new_type(field: &FieldRef, new_type: DataType) -> FieldRef { + Arc::new(field.as_ref().clone().with_data_type(new_type)) +} + /// Transform a schema to use view types for Utf8 and Binary +/// +/// See [parquet::ParquetFormat::force_view_types] for details pub fn transform_schema_to_view(schema: &Schema) -> Schema { let transformed_fields: Vec> = schema .fields .iter() .map(|field| match field.data_type() { - DataType::Utf8 | DataType::LargeUtf8 => Arc::new( - Field::new(field.name(), DataType::Utf8View, field.is_nullable()) - .with_metadata(field.metadata().to_owned()), - ), - DataType::Binary | DataType::LargeBinary => Arc::new( - Field::new(field.name(), DataType::BinaryView, field.is_nullable()) - .with_metadata(field.metadata().to_owned()), - ), + DataType::Utf8 | DataType::LargeUtf8 => { + field_with_new_type(field, DataType::Utf8View) + } + DataType::Binary | DataType::LargeBinary => { + field_with_new_type(field, DataType::BinaryView) + } _ => field.clone(), }) .collect(); @@ -274,6 +280,7 @@ pub(crate) fn coerce_file_schema_to_view_type( (f.name(), dt) }) .collect(); + if !transform { return None; } @@ -283,14 +290,13 @@ pub(crate) fn coerce_file_schema_to_view_type( .iter() .map( |field| match (table_fields.get(field.name()), field.data_type()) { - (Some(DataType::Utf8View), DataType::Utf8) - | (Some(DataType::Utf8View), DataType::LargeUtf8) => Arc::new( - Field::new(field.name(), DataType::Utf8View, field.is_nullable()), - ), - (Some(DataType::BinaryView), DataType::Binary) - | (Some(DataType::BinaryView), DataType::LargeBinary) => Arc::new( - Field::new(field.name(), DataType::BinaryView, field.is_nullable()), - ), + (Some(DataType::Utf8View), DataType::Utf8 | DataType::LargeUtf8) => { + field_with_new_type(field, DataType::Utf8View) + } + ( + Some(DataType::BinaryView), + DataType::Binary | DataType::LargeBinary, + ) => field_with_new_type(field, DataType::BinaryView), _ => field.clone(), }, ) @@ -302,6 +308,78 @@ pub(crate) fn coerce_file_schema_to_view_type( )) } +/// Transform a schema so that any binary types are strings +pub fn transform_binary_to_string(schema: &Schema) -> Schema { + let transformed_fields: Vec> = schema + .fields + .iter() + .map(|field| match field.data_type() { + DataType::Binary => field_with_new_type(field, DataType::Utf8), + DataType::LargeBinary => field_with_new_type(field, DataType::LargeUtf8), + DataType::BinaryView => field_with_new_type(field, DataType::Utf8View), + _ => field.clone(), + }) + .collect(); + Schema::new_with_metadata(transformed_fields, schema.metadata.clone()) +} + +/// If the table schema uses a string type, coerce the file schema to use a string type. +/// +/// See [parquet::ParquetFormat::binary_as_string] for details +pub(crate) fn coerce_file_schema_to_string_type( + table_schema: &Schema, + file_schema: &Schema, +) -> Option { + let mut transform = false; + let table_fields: HashMap<_, _> = table_schema + .fields + .iter() + .map(|f| (f.name(), f.data_type())) + .collect(); + let transformed_fields: Vec> = file_schema + .fields + .iter() + .map( + |field| match (table_fields.get(field.name()), field.data_type()) { + // table schema uses string type, coerce the file schema to use string type + ( + Some(DataType::Utf8), + DataType::Binary | DataType::LargeBinary | DataType::BinaryView, + ) => { + transform = true; + field_with_new_type(field, DataType::Utf8) + } + // table schema uses large string type, coerce the file schema to use large string type + ( + Some(DataType::LargeUtf8), + DataType::Binary | DataType::LargeBinary | DataType::BinaryView, + ) => { + transform = true; + field_with_new_type(field, DataType::LargeUtf8) + } + // table schema uses string view type, coerce the file schema to use view type + ( + Some(DataType::Utf8View), + DataType::Binary | DataType::LargeBinary | DataType::BinaryView, + ) => { + transform = true; + field_with_new_type(field, DataType::Utf8View) + } + _ => field.clone(), + }, + ) + .collect(); + + if !transform { + None + } else { + Some(Schema::new_with_metadata( + transformed_fields, + file_schema.metadata.clone(), + )) + } +} + #[cfg(test)] pub(crate) mod test_util { use std::ops::Range; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 8647b5df90be..756c17fd67c6 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -26,8 +26,9 @@ use std::sync::Arc; use super::write::demux::start_demuxer_task; use super::write::{create_writer, SharedBuffer}; use super::{ - coerce_file_schema_to_view_type, transform_schema_to_view, FileFormat, - FileFormatFactory, FilePushdownSupport, FileScanConfig, + coerce_file_schema_to_string_type, coerce_file_schema_to_view_type, + transform_binary_to_string, transform_schema_to_view, FileFormat, FileFormatFactory, + FilePushdownSupport, FileScanConfig, }; use crate::arrow::array::RecordBatch; use crate::arrow::datatypes::{Fields, Schema, SchemaRef}; @@ -253,13 +254,29 @@ impl ParquetFormat { self.options.global.schema_force_view_types } - /// If true, will use view types (StringView and BinaryView). - /// - /// Refer to [`Self::force_view_types`]. + /// If true, will use view types. See [`Self::force_view_types`] for details pub fn with_force_view_types(mut self, use_views: bool) -> Self { self.options.global.schema_force_view_types = use_views; self } + + /// Return `true` if binary types will be read as strings. + /// + /// If this returns true, DataFusion will instruct the parquet reader + /// to read binary columns such as `Binary` or `BinaryView` as the + /// corresponding string type such as `Utf8` or `LargeUtf8`. + /// The parquet reader has special optimizations for `Utf8` and `LargeUtf8` + /// validation, and such queries are significantly faster than reading + /// binary columns and then casting to string columns. + pub fn binary_as_string(&self) -> bool { + self.options.global.binary_as_string + } + + /// If true, will read binary types as strings. See [`Self::binary_as_string`] for details + pub fn with_binary_as_string(mut self, binary_as_string: bool) -> Self { + self.options.global.binary_as_string = binary_as_string; + self + } } /// Clears all metadata (Schema level and field level) on an iterator @@ -350,6 +367,12 @@ impl FileFormat for ParquetFormat { Schema::try_merge(schemas) }?; + let schema = if self.binary_as_string() { + transform_binary_to_string(&schema) + } else { + schema + }; + let schema = if self.force_view_types() { transform_schema_to_view(&schema) } else { @@ -552,6 +575,10 @@ pub fn statistics_from_parquet_meta_calc( file_metadata.schema_descr(), file_metadata.key_value_metadata(), )?; + if let Some(merged) = coerce_file_schema_to_string_type(&table_schema, &file_schema) { + file_schema = merged; + } + if let Some(merged) = coerce_file_schema_to_view_type(&table_schema, &file_schema) { file_schema = merged; } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs index a818a8850284..4990cb4dd735 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs @@ -17,7 +17,9 @@ //! [`ParquetOpener`] for opening Parquet files -use crate::datasource::file_format::coerce_file_schema_to_view_type; +use crate::datasource::file_format::{ + coerce_file_schema_to_string_type, coerce_file_schema_to_view_type, +}; use crate::datasource::physical_plan::parquet::page_filter::PagePruningAccessPlanFilter; use crate::datasource::physical_plan::parquet::row_group_filter::RowGroupAccessPlanFilter; use crate::datasource::physical_plan::parquet::{ @@ -80,7 +82,7 @@ pub(super) struct ParquetOpener { } impl FileOpener for ParquetOpener { - fn open(&self, file_meta: FileMeta) -> datafusion_common::Result { + fn open(&self, file_meta: FileMeta) -> Result { let file_range = file_meta.range.clone(); let extensions = file_meta.extensions.clone(); let file_name = file_meta.location().to_string(); @@ -121,7 +123,14 @@ impl FileOpener for ParquetOpener { let mut metadata_timer = file_metrics.metadata_load_time.timer(); let metadata = ArrowReaderMetadata::load_async(&mut reader, options.clone()).await?; - let mut schema = metadata.schema().clone(); + let mut schema = Arc::clone(metadata.schema()); + + if let Some(merged) = + coerce_file_schema_to_string_type(&table_schema, &schema) + { + schema = Arc::new(merged); + } + // read with view types if let Some(merged) = coerce_file_schema_to_view_type(&table_schema, &schema) { @@ -130,16 +139,16 @@ impl FileOpener for ParquetOpener { let options = ArrowReaderOptions::new() .with_page_index(enable_page_index) - .with_schema(schema.clone()); + .with_schema(Arc::clone(&schema)); let metadata = - ArrowReaderMetadata::try_new(metadata.metadata().clone(), options)?; + ArrowReaderMetadata::try_new(Arc::clone(metadata.metadata()), options)?; metadata_timer.stop(); let mut builder = ParquetRecordBatchStreamBuilder::new_with_metadata(reader, metadata); - let file_schema = builder.schema().clone(); + let file_schema = Arc::clone(builder.schema()); let (schema_mapping, adapted_projections) = schema_adapter.map_schema(&file_schema)?; @@ -177,7 +186,7 @@ impl FileOpener for ParquetOpener { // Determine which row groups to actually read. The idea is to skip // as many row groups as possible based on the metadata and query - let file_metadata = builder.metadata().clone(); + let file_metadata = Arc::clone(builder.metadata()); let predicate = pruning_predicate.as_ref().map(|p| p.as_ref()); let rg_metadata = file_metadata.row_groups(); // track which row groups to actually read diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index d1506fcd64f0..7f8bce6b206e 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -494,6 +494,7 @@ message ParquetOptions { bool bloom_filter_on_read = 26; // default = true bool bloom_filter_on_write = 27; // default = false bool schema_force_view_types = 28; // default = false + bool binary_as_string = 29; // default = false oneof metadata_size_hint_opt { uint64 metadata_size_hint = 4; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index d1b4374fc0e7..d848f795c684 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -897,7 +897,7 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { pruning: value.pruning, skip_metadata: value.skip_metadata, metadata_size_hint: value - .metadata_size_hint_opt.clone() + .metadata_size_hint_opt .map(|opt| match opt { protobuf::parquet_options::MetadataSizeHintOpt::MetadataSizeHint(v) => Some(v as usize), }) @@ -958,6 +958,7 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { maximum_parallel_row_group_writers: value.maximum_parallel_row_group_writers as usize, maximum_buffered_record_batches_per_stream: value.maximum_buffered_record_batches_per_stream as usize, schema_force_view_types: value.schema_force_view_types, + binary_as_string: value.binary_as_string, }) } } @@ -979,7 +980,7 @@ impl TryFrom<&protobuf::ParquetColumnOptions> for ParquetColumnOptions { }) .unwrap_or(None), max_statistics_size: value - .max_statistics_size_opt.clone() + .max_statistics_size_opt .map(|opt| match opt { protobuf::parquet_column_options::MaxStatisticsSizeOpt::MaxStatisticsSize(v) => Some(v as usize), }) @@ -990,18 +991,18 @@ impl TryFrom<&protobuf::ParquetColumnOptions> for ParquetColumnOptions { protobuf::parquet_column_options::EncodingOpt::Encoding(v) => Some(v), }) .unwrap_or(None), - bloom_filter_enabled: value.bloom_filter_enabled_opt.clone().map(|opt| match opt { + bloom_filter_enabled: value.bloom_filter_enabled_opt.map(|opt| match opt { protobuf::parquet_column_options::BloomFilterEnabledOpt::BloomFilterEnabled(v) => Some(v), }) .unwrap_or(None), bloom_filter_fpp: value - .bloom_filter_fpp_opt.clone() + .bloom_filter_fpp_opt .map(|opt| match opt { protobuf::parquet_column_options::BloomFilterFppOpt::BloomFilterFpp(v) => Some(v), }) .unwrap_or(None), bloom_filter_ndv: value - .bloom_filter_ndv_opt.clone() + .bloom_filter_ndv_opt .map(|opt| match opt { protobuf::parquet_column_options::BloomFilterNdvOpt::BloomFilterNdv(v) => Some(v), }) diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index fa5d1f442754..e8b46fbf7012 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -1548,18 +1548,22 @@ impl serde::Serialize for CsvOptions { let mut struct_ser = serializer.serialize_struct("datafusion_common.CsvOptions", len)?; if !self.has_header.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("hasHeader", pbjson::private::base64::encode(&self.has_header).as_str())?; } if !self.delimiter.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("delimiter", pbjson::private::base64::encode(&self.delimiter).as_str())?; } if !self.quote.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("quote", pbjson::private::base64::encode(&self.quote).as_str())?; } if !self.escape.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("escape", pbjson::private::base64::encode(&self.escape).as_str())?; } if self.compression != 0 { @@ -1569,6 +1573,7 @@ impl serde::Serialize for CsvOptions { } if self.schema_infer_max_rec != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("schemaInferMaxRec", ToString::to_string(&self.schema_infer_max_rec).as_str())?; } if !self.date_format.is_empty() { @@ -1591,18 +1596,22 @@ impl serde::Serialize for CsvOptions { } if !self.comment.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("comment", pbjson::private::base64::encode(&self.comment).as_str())?; } if !self.double_quote.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("doubleQuote", pbjson::private::base64::encode(&self.double_quote).as_str())?; } if !self.newlines_in_values.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("newlinesInValues", pbjson::private::base64::encode(&self.newlines_in_values).as_str())?; } if !self.terminator.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("terminator", pbjson::private::base64::encode(&self.terminator).as_str())?; } struct_ser.end() @@ -2276,14 +2285,17 @@ impl serde::Serialize for Decimal128 { let mut struct_ser = serializer.serialize_struct("datafusion_common.Decimal128", len)?; if !self.value.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("value", pbjson::private::base64::encode(&self.value).as_str())?; } if self.p != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("p", ToString::to_string(&self.p).as_str())?; } if self.s != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("s", ToString::to_string(&self.s).as_str())?; } struct_ser.end() @@ -2410,14 +2422,17 @@ impl serde::Serialize for Decimal256 { let mut struct_ser = serializer.serialize_struct("datafusion_common.Decimal256", len)?; if !self.value.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("value", pbjson::private::base64::encode(&self.value).as_str())?; } if self.p != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("p", ToString::to_string(&self.p).as_str())?; } if self.s != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("s", ToString::to_string(&self.s).as_str())?; } struct_ser.end() @@ -3080,6 +3095,7 @@ impl serde::Serialize for Field { } if self.dict_id != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("dictId", ToString::to_string(&self.dict_id).as_str())?; } if self.dict_ordered { @@ -3484,6 +3500,7 @@ impl serde::Serialize for IntervalMonthDayNanoValue { } if self.nanos != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("nanos", ToString::to_string(&self.nanos).as_str())?; } struct_ser.end() @@ -3917,6 +3934,7 @@ impl serde::Serialize for JsonOptions { } if self.schema_infer_max_rec != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("schemaInferMaxRec", ToString::to_string(&self.schema_infer_max_rec).as_str())?; } struct_ser.end() @@ -4474,6 +4492,7 @@ impl serde::Serialize for ParquetColumnOptions { match v { parquet_column_options::BloomFilterNdvOpt::BloomFilterNdv(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("bloomFilterNdv", ToString::to_string(&v).as_str())?; } } @@ -4894,6 +4913,9 @@ impl serde::Serialize for ParquetOptions { if self.schema_force_view_types { len += 1; } + if self.binary_as_string { + len += 1; + } if self.dictionary_page_size_limit != 0 { len += 1; } @@ -4951,10 +4973,12 @@ impl serde::Serialize for ParquetOptions { } if self.data_pagesize_limit != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("dataPagesizeLimit", ToString::to_string(&self.data_pagesize_limit).as_str())?; } if self.write_batch_size != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("writeBatchSize", ToString::to_string(&self.write_batch_size).as_str())?; } if !self.writer_version.is_empty() { @@ -4965,10 +4989,12 @@ impl serde::Serialize for ParquetOptions { } if self.maximum_parallel_row_group_writers != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("maximumParallelRowGroupWriters", ToString::to_string(&self.maximum_parallel_row_group_writers).as_str())?; } if self.maximum_buffered_record_batches_per_stream != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("maximumBufferedRecordBatchesPerStream", ToString::to_string(&self.maximum_buffered_record_batches_per_stream).as_str())?; } if self.bloom_filter_on_read { @@ -4980,16 +5006,22 @@ impl serde::Serialize for ParquetOptions { if self.schema_force_view_types { struct_ser.serialize_field("schemaForceViewTypes", &self.schema_force_view_types)?; } + if self.binary_as_string { + struct_ser.serialize_field("binaryAsString", &self.binary_as_string)?; + } if self.dictionary_page_size_limit != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("dictionaryPageSizeLimit", ToString::to_string(&self.dictionary_page_size_limit).as_str())?; } if self.data_page_row_count_limit != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("dataPageRowCountLimit", ToString::to_string(&self.data_page_row_count_limit).as_str())?; } if self.max_row_group_size != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("maxRowGroupSize", ToString::to_string(&self.max_row_group_size).as_str())?; } if !self.created_by.is_empty() { @@ -4999,6 +5031,7 @@ impl serde::Serialize for ParquetOptions { match v { parquet_options::MetadataSizeHintOpt::MetadataSizeHint(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("metadataSizeHint", ToString::to_string(&v).as_str())?; } } @@ -5028,6 +5061,7 @@ impl serde::Serialize for ParquetOptions { match v { parquet_options::MaxStatisticsSizeOpt::MaxStatisticsSize(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("maxStatisticsSize", ToString::to_string(&v).as_str())?; } } @@ -5036,6 +5070,7 @@ impl serde::Serialize for ParquetOptions { match v { parquet_options::ColumnIndexTruncateLengthOpt::ColumnIndexTruncateLength(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("columnIndexTruncateLength", ToString::to_string(&v).as_str())?; } } @@ -5058,6 +5093,7 @@ impl serde::Serialize for ParquetOptions { match v { parquet_options::BloomFilterNdvOpt::BloomFilterNdv(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("bloomFilterNdv", ToString::to_string(&v).as_str())?; } } @@ -5099,6 +5135,8 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "bloomFilterOnWrite", "schema_force_view_types", "schemaForceViewTypes", + "binary_as_string", + "binaryAsString", "dictionary_page_size_limit", "dictionaryPageSizeLimit", "data_page_row_count_limit", @@ -5140,7 +5178,8 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { MaximumBufferedRecordBatchesPerStream, BloomFilterOnRead, BloomFilterOnWrite, - schemaForceViewTypes, + SchemaForceViewTypes, + BinaryAsString, DictionaryPageSizeLimit, DataPageRowCountLimit, MaxRowGroupSize, @@ -5188,7 +5227,8 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "maximumBufferedRecordBatchesPerStream" | "maximum_buffered_record_batches_per_stream" => Ok(GeneratedField::MaximumBufferedRecordBatchesPerStream), "bloomFilterOnRead" | "bloom_filter_on_read" => Ok(GeneratedField::BloomFilterOnRead), "bloomFilterOnWrite" | "bloom_filter_on_write" => Ok(GeneratedField::BloomFilterOnWrite), - "schemaForceViewTypes" | "schema_force_view_types" => Ok(GeneratedField::schemaForceViewTypes), + "schemaForceViewTypes" | "schema_force_view_types" => Ok(GeneratedField::SchemaForceViewTypes), + "binaryAsString" | "binary_as_string" => Ok(GeneratedField::BinaryAsString), "dictionaryPageSizeLimit" | "dictionary_page_size_limit" => Ok(GeneratedField::DictionaryPageSizeLimit), "dataPageRowCountLimit" | "data_page_row_count_limit" => Ok(GeneratedField::DataPageRowCountLimit), "maxRowGroupSize" | "max_row_group_size" => Ok(GeneratedField::MaxRowGroupSize), @@ -5235,6 +5275,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { let mut bloom_filter_on_read__ = None; let mut bloom_filter_on_write__ = None; let mut schema_force_view_types__ = None; + let mut binary_as_string__ = None; let mut dictionary_page_size_limit__ = None; let mut data_page_row_count_limit__ = None; let mut max_row_group_size__ = None; @@ -5336,12 +5377,18 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { } bloom_filter_on_write__ = Some(map_.next_value()?); } - GeneratedField::schemaForceViewTypes => { + GeneratedField::SchemaForceViewTypes => { if schema_force_view_types__.is_some() { return Err(serde::de::Error::duplicate_field("schemaForceViewTypes")); } schema_force_view_types__ = Some(map_.next_value()?); } + GeneratedField::BinaryAsString => { + if binary_as_string__.is_some() { + return Err(serde::de::Error::duplicate_field("binaryAsString")); + } + binary_as_string__ = Some(map_.next_value()?); + } GeneratedField::DictionaryPageSizeLimit => { if dictionary_page_size_limit__.is_some() { return Err(serde::de::Error::duplicate_field("dictionaryPageSizeLimit")); @@ -5443,6 +5490,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { bloom_filter_on_read: bloom_filter_on_read__.unwrap_or_default(), bloom_filter_on_write: bloom_filter_on_write__.unwrap_or_default(), schema_force_view_types: schema_force_view_types__.unwrap_or_default(), + binary_as_string: binary_as_string__.unwrap_or_default(), dictionary_page_size_limit: dictionary_page_size_limit__.unwrap_or_default(), data_page_row_count_limit: data_page_row_count_limit__.unwrap_or_default(), max_row_group_size: max_row_group_size__.unwrap_or_default(), @@ -5867,6 +5915,7 @@ impl serde::Serialize for ScalarFixedSizeBinary { let mut struct_ser = serializer.serialize_struct("datafusion_common.ScalarFixedSizeBinary", len)?; if !self.values.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("values", pbjson::private::base64::encode(&self.values).as_str())?; } if self.length != 0 { @@ -5986,10 +6035,12 @@ impl serde::Serialize for ScalarNestedValue { let mut struct_ser = serializer.serialize_struct("datafusion_common.ScalarNestedValue", len)?; if !self.ipc_message.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("ipcMessage", pbjson::private::base64::encode(&self.ipc_message).as_str())?; } if !self.arrow_data.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("arrowData", pbjson::private::base64::encode(&self.arrow_data).as_str())?; } if let Some(v) = self.schema.as_ref() { @@ -6130,10 +6181,12 @@ impl serde::Serialize for scalar_nested_value::Dictionary { let mut struct_ser = serializer.serialize_struct("datafusion_common.ScalarNestedValue.Dictionary", len)?; if !self.ipc_message.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("ipcMessage", pbjson::private::base64::encode(&self.ipc_message).as_str())?; } if !self.arrow_data.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("arrowData", pbjson::private::base64::encode(&self.arrow_data).as_str())?; } struct_ser.end() @@ -6354,10 +6407,12 @@ impl serde::Serialize for ScalarTime64Value { match v { scalar_time64_value::Value::Time64MicrosecondValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("time64MicrosecondValue", ToString::to_string(&v).as_str())?; } scalar_time64_value::Value::Time64NanosecondValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("time64NanosecondValue", ToString::to_string(&v).as_str())?; } } @@ -6471,18 +6526,22 @@ impl serde::Serialize for ScalarTimestampValue { match v { scalar_timestamp_value::Value::TimeMicrosecondValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("timeMicrosecondValue", ToString::to_string(&v).as_str())?; } scalar_timestamp_value::Value::TimeNanosecondValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("timeNanosecondValue", ToString::to_string(&v).as_str())?; } scalar_timestamp_value::Value::TimeSecondValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("timeSecondValue", ToString::to_string(&v).as_str())?; } scalar_timestamp_value::Value::TimeMillisecondValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("timeMillisecondValue", ToString::to_string(&v).as_str())?; } } @@ -6645,6 +6704,7 @@ impl serde::Serialize for ScalarValue { } scalar_value::Value::Int64Value(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("int64Value", ToString::to_string(&v).as_str())?; } scalar_value::Value::Uint8Value(v) => { @@ -6658,6 +6718,7 @@ impl serde::Serialize for ScalarValue { } scalar_value::Value::Uint64Value(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("uint64Value", ToString::to_string(&v).as_str())?; } scalar_value::Value::Float32Value(v) => { @@ -6695,6 +6756,7 @@ impl serde::Serialize for ScalarValue { } scalar_value::Value::Date64Value(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("date64Value", ToString::to_string(&v).as_str())?; } scalar_value::Value::IntervalYearmonthValue(v) => { @@ -6702,18 +6764,22 @@ impl serde::Serialize for ScalarValue { } scalar_value::Value::DurationSecondValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("durationSecondValue", ToString::to_string(&v).as_str())?; } scalar_value::Value::DurationMillisecondValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("durationMillisecondValue", ToString::to_string(&v).as_str())?; } scalar_value::Value::DurationMicrosecondValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("durationMicrosecondValue", ToString::to_string(&v).as_str())?; } scalar_value::Value::DurationNanosecondValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("durationNanosecondValue", ToString::to_string(&v).as_str())?; } scalar_value::Value::TimestampValue(v) => { @@ -6724,14 +6790,17 @@ impl serde::Serialize for ScalarValue { } scalar_value::Value::BinaryValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("binaryValue", pbjson::private::base64::encode(&v).as_str())?; } scalar_value::Value::LargeBinaryValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("largeBinaryValue", pbjson::private::base64::encode(&v).as_str())?; } scalar_value::Value::BinaryViewValue(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("binaryViewValue", pbjson::private::base64::encode(&v).as_str())?; } scalar_value::Value::Time64Value(v) => { diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index d6f982278d67..939a4b3c2cd2 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -1,11 +1,9 @@ // This file is @generated by prost-build. -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ColumnRelation { #[prost(string, tag = "1")] pub relation: ::prost::alloc::string::String, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Column { #[prost(string, tag = "1")] @@ -13,7 +11,6 @@ pub struct Column { #[prost(message, optional, tag = "2")] pub relation: ::core::option::Option, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DfField { #[prost(message, optional, tag = "1")] @@ -21,7 +18,6 @@ pub struct DfField { #[prost(message, optional, tag = "2")] pub qualifier: ::core::option::Option, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DfSchema { #[prost(message, repeated, tag = "1")] @@ -32,40 +28,33 @@ pub struct DfSchema { ::prost::alloc::string::String, >, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CsvFormat { #[prost(message, optional, tag = "5")] pub options: ::core::option::Option, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ParquetFormat { #[prost(message, optional, tag = "2")] pub options: ::core::option::Option, } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct AvroFormat {} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct NdJsonFormat { #[prost(message, optional, tag = "1")] pub options: ::core::option::Option, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct PrimaryKeyConstraint { #[prost(uint64, repeated, tag = "1")] pub indices: ::prost::alloc::vec::Vec, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct UniqueConstraint { #[prost(uint64, repeated, tag = "1")] pub indices: ::prost::alloc::vec::Vec, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Constraint { #[prost(oneof = "constraint::ConstraintMode", tags = "1, 2")] @@ -73,7 +62,6 @@ pub struct Constraint { } /// Nested message and enum types in `Constraint`. pub mod constraint { - #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum ConstraintMode { #[prost(message, tag = "1")] @@ -82,19 +70,15 @@ pub mod constraint { Unique(super::UniqueConstraint), } } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Constraints { #[prost(message, repeated, tag = "1")] pub constraints: ::prost::alloc::vec::Vec, } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct AvroOptions {} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ArrowOptions {} -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Schema { #[prost(message, repeated, tag = "1")] @@ -105,7 +89,6 @@ pub struct Schema { ::prost::alloc::string::String, >, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Field { /// name of the field @@ -128,7 +111,6 @@ pub struct Field { #[prost(bool, tag = "7")] pub dict_ordered: bool, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Timestamp { #[prost(enumeration = "TimeUnit", tag = "1")] @@ -136,29 +118,25 @@ pub struct Timestamp { #[prost(string, tag = "2")] pub timezone: ::prost::alloc::string::String, } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct Decimal { #[prost(uint32, tag = "3")] pub precision: u32, #[prost(int32, tag = "4")] pub scale: i32, } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct Decimal256Type { #[prost(uint32, tag = "3")] pub precision: u32, #[prost(int32, tag = "4")] pub scale: i32, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct List { #[prost(message, optional, boxed, tag = "1")] pub field_type: ::core::option::Option<::prost::alloc::boxed::Box>, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct FixedSizeList { #[prost(message, optional, boxed, tag = "1")] @@ -166,7 +144,6 @@ pub struct FixedSizeList { #[prost(int32, tag = "2")] pub list_size: i32, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Dictionary { #[prost(message, optional, boxed, tag = "1")] @@ -174,13 +151,11 @@ pub struct Dictionary { #[prost(message, optional, boxed, tag = "2")] pub value: ::core::option::Option<::prost::alloc::boxed::Box>, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Struct { #[prost(message, repeated, tag = "1")] pub sub_field_types: ::prost::alloc::vec::Vec, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Map { #[prost(message, optional, boxed, tag = "1")] @@ -188,7 +163,6 @@ pub struct Map { #[prost(bool, tag = "2")] pub keys_sorted: bool, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Union { #[prost(message, repeated, tag = "1")] @@ -199,7 +173,6 @@ pub struct Union { pub type_ids: ::prost::alloc::vec::Vec, } /// Used for List/FixedSizeList/LargeList/Struct/Map -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ScalarNestedValue { #[prost(bytes = "vec", tag = "1")] @@ -213,7 +186,6 @@ pub struct ScalarNestedValue { } /// Nested message and enum types in `ScalarNestedValue`. pub mod scalar_nested_value { - #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Dictionary { #[prost(bytes = "vec", tag = "1")] @@ -222,16 +194,14 @@ pub mod scalar_nested_value { pub arrow_data: ::prost::alloc::vec::Vec, } } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ScalarTime32Value { #[prost(oneof = "scalar_time32_value::Value", tags = "1, 2")] pub value: ::core::option::Option, } /// Nested message and enum types in `ScalarTime32Value`. pub mod scalar_time32_value { - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum Value { #[prost(int32, tag = "1")] Time32SecondValue(i32), @@ -239,16 +209,14 @@ pub mod scalar_time32_value { Time32MillisecondValue(i32), } } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ScalarTime64Value { #[prost(oneof = "scalar_time64_value::Value", tags = "1, 2")] pub value: ::core::option::Option, } /// Nested message and enum types in `ScalarTime64Value`. pub mod scalar_time64_value { - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum Value { #[prost(int64, tag = "1")] Time64MicrosecondValue(i64), @@ -256,7 +224,6 @@ pub mod scalar_time64_value { Time64NanosecondValue(i64), } } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ScalarTimestampValue { #[prost(string, tag = "5")] @@ -266,8 +233,7 @@ pub struct ScalarTimestampValue { } /// Nested message and enum types in `ScalarTimestampValue`. pub mod scalar_timestamp_value { - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum Value { #[prost(int64, tag = "1")] TimeMicrosecondValue(i64), @@ -279,7 +245,6 @@ pub mod scalar_timestamp_value { TimeMillisecondValue(i64), } } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ScalarDictionaryValue { #[prost(message, optional, tag = "1")] @@ -287,16 +252,14 @@ pub struct ScalarDictionaryValue { #[prost(message, optional, boxed, tag = "2")] pub value: ::core::option::Option<::prost::alloc::boxed::Box>, } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct IntervalDayTimeValue { #[prost(int32, tag = "1")] pub days: i32, #[prost(int32, tag = "2")] pub milliseconds: i32, } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct IntervalMonthDayNanoValue { #[prost(int32, tag = "1")] pub months: i32, @@ -305,7 +268,6 @@ pub struct IntervalMonthDayNanoValue { #[prost(int64, tag = "3")] pub nanos: i64, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct UnionField { #[prost(int32, tag = "1")] @@ -313,7 +275,6 @@ pub struct UnionField { #[prost(message, optional, tag = "2")] pub field: ::core::option::Option, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct UnionValue { /// Note that a null union value must have one or more fields, so we @@ -327,7 +288,6 @@ pub struct UnionValue { #[prost(enumeration = "UnionMode", tag = "4")] pub mode: i32, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ScalarFixedSizeBinary { #[prost(bytes = "vec", tag = "1")] @@ -335,7 +295,6 @@ pub struct ScalarFixedSizeBinary { #[prost(int32, tag = "2")] pub length: i32, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ScalarValue { #[prost( @@ -346,7 +305,6 @@ pub struct ScalarValue { } /// Nested message and enum types in `ScalarValue`. pub mod scalar_value { - #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum Value { /// was PrimitiveScalarType null_value = 19; @@ -434,7 +392,6 @@ pub mod scalar_value { UnionValue(::prost::alloc::boxed::Box), } } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Decimal128 { #[prost(bytes = "vec", tag = "1")] @@ -444,7 +401,6 @@ pub struct Decimal128 { #[prost(int64, tag = "3")] pub s: i64, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Decimal256 { #[prost(bytes = "vec", tag = "1")] @@ -455,7 +411,6 @@ pub struct Decimal256 { pub s: i64, } /// Serialized data type -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ArrowType { #[prost( @@ -466,7 +421,6 @@ pub struct ArrowType { } /// Nested message and enum types in `ArrowType`. pub mod arrow_type { - #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum ArrowTypeEnum { /// arrow::Type::NA @@ -557,16 +511,13 @@ pub mod arrow_type { /// i32 Two = 2; /// } /// } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct EmptyMessage {} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct JsonWriterOptions { #[prost(enumeration = "CompressionTypeVariant", tag = "1")] pub compression: i32, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CsvWriterOptions { /// Compression type @@ -604,7 +555,6 @@ pub struct CsvWriterOptions { pub double_quote: bool, } /// Options controlling CSV format -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CsvOptions { /// Indicates if the CSV has a header row @@ -657,8 +607,7 @@ pub struct CsvOptions { pub terminator: ::prost::alloc::vec::Vec, } /// Options controlling CSV format -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct JsonOptions { /// Compression type #[prost(enumeration = "CompressionTypeVariant", tag = "1")] @@ -667,7 +616,6 @@ pub struct JsonOptions { #[prost(uint64, tag = "2")] pub schema_infer_max_rec: u64, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct TableParquetOptions { #[prost(message, optional, tag = "1")] @@ -680,7 +628,6 @@ pub struct TableParquetOptions { ::prost::alloc::string::String, >, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ParquetColumnSpecificOptions { #[prost(string, tag = "1")] @@ -688,7 +635,6 @@ pub struct ParquetColumnSpecificOptions { #[prost(message, optional, tag = "2")] pub options: ::core::option::Option, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ParquetColumnOptions { #[prost(oneof = "parquet_column_options::BloomFilterEnabledOpt", tags = "1")] @@ -722,56 +668,47 @@ pub struct ParquetColumnOptions { } /// Nested message and enum types in `ParquetColumnOptions`. pub mod parquet_column_options { - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterEnabledOpt { #[prost(bool, tag = "1")] BloomFilterEnabled(bool), } - #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum EncodingOpt { #[prost(string, tag = "2")] Encoding(::prost::alloc::string::String), } - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum DictionaryEnabledOpt { #[prost(bool, tag = "3")] DictionaryEnabled(bool), } - #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum CompressionOpt { #[prost(string, tag = "4")] Compression(::prost::alloc::string::String), } - #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum StatisticsEnabledOpt { #[prost(string, tag = "5")] StatisticsEnabled(::prost::alloc::string::String), } - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterFppOpt { #[prost(double, tag = "6")] BloomFilterFpp(f64), } - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterNdvOpt { #[prost(uint64, tag = "7")] BloomFilterNdv(u64), } - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum MaxStatisticsSizeOpt { #[prost(uint32, tag = "8")] MaxStatisticsSize(u32), } } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ParquetOptions { /// Regular fields @@ -820,6 +757,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "28")] pub schema_force_view_types: bool, + /// default = false + #[prost(bool, tag = "29")] + pub binary_as_string: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] @@ -859,62 +799,52 @@ pub struct ParquetOptions { } /// Nested message and enum types in `ParquetOptions`. pub mod parquet_options { - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum MetadataSizeHintOpt { #[prost(uint64, tag = "4")] MetadataSizeHint(u64), } - #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum CompressionOpt { #[prost(string, tag = "10")] Compression(::prost::alloc::string::String), } - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum DictionaryEnabledOpt { #[prost(bool, tag = "11")] DictionaryEnabled(bool), } - #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum StatisticsEnabledOpt { #[prost(string, tag = "13")] StatisticsEnabled(::prost::alloc::string::String), } - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum MaxStatisticsSizeOpt { #[prost(uint64, tag = "14")] MaxStatisticsSize(u64), } - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum ColumnIndexTruncateLengthOpt { #[prost(uint64, tag = "17")] ColumnIndexTruncateLength(u64), } - #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum EncodingOpt { #[prost(string, tag = "19")] Encoding(::prost::alloc::string::String), } - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterFppOpt { #[prost(double, tag = "21")] BloomFilterFpp(f64), } - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterNdvOpt { #[prost(uint64, tag = "22")] BloomFilterNdv(u64), } } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Precision { #[prost(enumeration = "PrecisionInfo", tag = "1")] @@ -922,7 +852,6 @@ pub struct Precision { #[prost(message, optional, tag = "2")] pub val: ::core::option::Option, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Statistics { #[prost(message, optional, tag = "1")] @@ -932,7 +861,6 @@ pub struct Statistics { #[prost(message, repeated, tag = "3")] pub column_stats: ::prost::alloc::vec::Vec, } -#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ColumnStats { #[prost(message, optional, tag = "1")] @@ -963,14 +891,14 @@ impl JoinType { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - JoinType::Inner => "INNER", - JoinType::Left => "LEFT", - JoinType::Right => "RIGHT", - JoinType::Full => "FULL", - JoinType::Leftsemi => "LEFTSEMI", - JoinType::Leftanti => "LEFTANTI", - JoinType::Rightsemi => "RIGHTSEMI", - JoinType::Rightanti => "RIGHTANTI", + Self::Inner => "INNER", + Self::Left => "LEFT", + Self::Right => "RIGHT", + Self::Full => "FULL", + Self::Leftsemi => "LEFTSEMI", + Self::Leftanti => "LEFTANTI", + Self::Rightsemi => "RIGHTSEMI", + Self::Rightanti => "RIGHTANTI", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -1001,8 +929,8 @@ impl JoinConstraint { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - JoinConstraint::On => "ON", - JoinConstraint::Using => "USING", + Self::On => "ON", + Self::Using => "USING", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -1029,10 +957,10 @@ impl TimeUnit { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - TimeUnit::Second => "Second", - TimeUnit::Millisecond => "Millisecond", - TimeUnit::Microsecond => "Microsecond", - TimeUnit::Nanosecond => "Nanosecond", + Self::Second => "Second", + Self::Millisecond => "Millisecond", + Self::Microsecond => "Microsecond", + Self::Nanosecond => "Nanosecond", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -1060,9 +988,9 @@ impl IntervalUnit { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - IntervalUnit::YearMonth => "YearMonth", - IntervalUnit::DayTime => "DayTime", - IntervalUnit::MonthDayNano => "MonthDayNano", + Self::YearMonth => "YearMonth", + Self::DayTime => "DayTime", + Self::MonthDayNano => "MonthDayNano", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -1088,8 +1016,8 @@ impl UnionMode { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - UnionMode::Sparse => "sparse", - UnionMode::Dense => "dense", + Self::Sparse => "sparse", + Self::Dense => "dense", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -1117,11 +1045,11 @@ impl CompressionTypeVariant { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - CompressionTypeVariant::Gzip => "GZIP", - CompressionTypeVariant::Bzip2 => "BZIP2", - CompressionTypeVariant::Xz => "XZ", - CompressionTypeVariant::Zstd => "ZSTD", - CompressionTypeVariant::Uncompressed => "UNCOMPRESSED", + Self::Gzip => "GZIP", + Self::Bzip2 => "BZIP2", + Self::Xz => "XZ", + Self::Zstd => "ZSTD", + Self::Uncompressed => "UNCOMPRESSED", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -1149,8 +1077,8 @@ impl JoinSide { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - JoinSide::LeftSide => "LEFT_SIDE", - JoinSide::RightSide => "RIGHT_SIDE", + Self::LeftSide => "LEFT_SIDE", + Self::RightSide => "RIGHT_SIDE", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -1176,9 +1104,9 @@ impl PrecisionInfo { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - PrecisionInfo::Exact => "EXACT", - PrecisionInfo::Inexact => "INEXACT", - PrecisionInfo::Absent => "ABSENT", + Self::Exact => "EXACT", + Self::Inexact => "INEXACT", + Self::Absent => "ABSENT", } } /// Creates an enum from field names used in the ProtoBuf definition. diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index ebb53ae7577c..f9b8973e2d41 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -831,6 +831,7 @@ impl TryFrom<&ParquetOptions> for protobuf::ParquetOptions { maximum_parallel_row_group_writers: value.maximum_parallel_row_group_writers as u64, maximum_buffered_record_batches_per_stream: value.maximum_buffered_record_batches_per_stream as u64, schema_force_view_types: value.schema_force_view_types, + binary_as_string: value.binary_as_string, }) } } diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index 16de2c777241..939a4b3c2cd2 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -757,6 +757,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "28")] pub schema_force_view_types: bool, + /// default = false + #[prost(bool, tag = "29")] + pub binary_as_string: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 98034e3082af..d0f82ecac62c 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -403,6 +403,7 @@ impl TableParquetOptionsProto { maximum_parallel_row_group_writers: global_options.global.maximum_parallel_row_group_writers as u64, maximum_buffered_record_batches_per_stream: global_options.global.maximum_buffered_record_batches_per_stream as u64, schema_force_view_types: global_options.global.schema_force_view_types, + binary_as_string: global_options.global.binary_as_string, }), column_specific_options: column_specific_options.into_iter().map(|(column_name, options)| { ParquetColumnSpecificOptions { @@ -493,6 +494,7 @@ impl From<&ParquetOptionsProto> for ParquetOptions { maximum_parallel_row_group_writers: proto.maximum_parallel_row_group_writers as usize, maximum_buffered_record_batches_per_stream: proto.maximum_buffered_record_batches_per_stream as usize, schema_force_view_types: proto.schema_force_view_types, + binary_as_string: proto.binary_as_string, } } } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 57bf029a63c1..3630f6c36595 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -180,6 +180,7 @@ datafusion.execution.max_buffered_batches_per_output_file 2 datafusion.execution.meta_fetch_concurrency 32 datafusion.execution.minimum_parallel_output_files 4 datafusion.execution.parquet.allow_single_file_parallelism true +datafusion.execution.parquet.binary_as_string false datafusion.execution.parquet.bloom_filter_fpp NULL datafusion.execution.parquet.bloom_filter_ndv NULL datafusion.execution.parquet.bloom_filter_on_read true @@ -271,6 +272,7 @@ datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. datafusion.execution.parquet.allow_single_file_parallelism true (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. +datafusion.execution.parquet.binary_as_string false (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. datafusion.execution.parquet.bloom_filter_fpp NULL (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting datafusion.execution.parquet.bloom_filter_ndv NULL (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting datafusion.execution.parquet.bloom_filter_on_read true (writing) Use any available bloom filters when reading parquet files diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index f8b163adc796..bf68a1851137 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -348,3 +348,204 @@ DROP TABLE list_columns; # Clean up statement ok DROP TABLE listing_table; + +### Tests for binary_ar_string + +# This scenario models the case where a column has been stored in parquet +# "binary" column (without a String logical type annotation) +# this is the case with the `hits_partitioned` ClickBench datasets +# see https://github.com/apache/datafusion/issues/12788 + +## Create a table with a binary column + +query I +COPY ( + SELECT + arrow_cast(string_col, 'Binary') as binary_col, + arrow_cast(string_col, 'LargeBinary') as largebinary_col, + arrow_cast(string_col, 'BinaryView') as binaryview_col + FROM src_table + ) +TO 'test_files/scratch/parquet/binary_as_string.parquet' +STORED AS PARQUET; +---- +9 + +# Test 1: Read table with default options +statement ok +CREATE EXTERNAL TABLE binary_as_string_default +STORED AS PARQUET LOCATION 'test_files/scratch/parquet/binary_as_string.parquet' + +# NB the data is read and displayed as binary +query T?T?T? +select + arrow_typeof(binary_col), binary_col, + arrow_typeof(largebinary_col), largebinary_col, + arrow_typeof(binaryview_col), binaryview_col + FROM binary_as_string_default; +---- +Binary 616161 Binary 616161 Binary 616161 +Binary 626262 Binary 626262 Binary 626262 +Binary 636363 Binary 636363 Binary 636363 +Binary 646464 Binary 646464 Binary 646464 +Binary 656565 Binary 656565 Binary 656565 +Binary 666666 Binary 666666 Binary 666666 +Binary 676767 Binary 676767 Binary 676767 +Binary 686868 Binary 686868 Binary 686868 +Binary 696969 Binary 696969 Binary 696969 + +# Run an explain plan to show the cast happens in the plan (a CAST is needed for the predicates) +query TT +EXPLAIN + SELECT binary_col, largebinary_col, binaryview_col + FROM binary_as_string_default + WHERE + binary_col LIKE '%a%' AND + largebinary_col LIKE '%a%' AND + binaryview_col LIKE '%a%'; +---- +logical_plan +01)Filter: CAST(binary_as_string_default.binary_col AS Utf8) LIKE Utf8("%a%") AND CAST(binary_as_string_default.largebinary_col AS Utf8) LIKE Utf8("%a%") AND CAST(binary_as_string_default.binaryview_col AS Utf8) LIKE Utf8("%a%") +02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8) LIKE Utf8("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8) LIKE Utf8("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8) LIKE Utf8("%a%")] +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: CAST(binary_col@0 AS Utf8) LIKE %a% AND CAST(largebinary_col@1 AS Utf8) LIKE %a% AND CAST(binaryview_col@2 AS Utf8) LIKE %a% +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], predicate=CAST(binary_col@0 AS Utf8) LIKE %a% AND CAST(largebinary_col@1 AS Utf8) LIKE %a% AND CAST(binaryview_col@2 AS Utf8) LIKE %a% + + +statement ok +DROP TABLE binary_as_string_default; + +## Test 2: Read table using the binary_as_string option + +statement ok +CREATE EXTERNAL TABLE binary_as_string_option +STORED AS PARQUET LOCATION 'test_files/scratch/parquet/binary_as_string.parquet' +OPTIONS ('binary_as_string' 'true'); + +# NB the data is read and displayed as string +query TTTTTT +select + arrow_typeof(binary_col), binary_col, + arrow_typeof(largebinary_col), largebinary_col, + arrow_typeof(binaryview_col), binaryview_col + FROM binary_as_string_option; +---- +Utf8 aaa Utf8 aaa Utf8 aaa +Utf8 bbb Utf8 bbb Utf8 bbb +Utf8 ccc Utf8 ccc Utf8 ccc +Utf8 ddd Utf8 ddd Utf8 ddd +Utf8 eee Utf8 eee Utf8 eee +Utf8 fff Utf8 fff Utf8 fff +Utf8 ggg Utf8 ggg Utf8 ggg +Utf8 hhh Utf8 hhh Utf8 hhh +Utf8 iii Utf8 iii Utf8 iii + +# Run an explain plan to show the cast happens in the plan (there should be no casts) +query TT +EXPLAIN + SELECT binary_col, largebinary_col, binaryview_col + FROM binary_as_string_option + WHERE + binary_col LIKE '%a%' AND + largebinary_col LIKE '%a%' AND + binaryview_col LIKE '%a%'; +---- +logical_plan +01)Filter: binary_as_string_option.binary_col LIKE Utf8("%a%") AND binary_as_string_option.largebinary_col LIKE Utf8("%a%") AND binary_as_string_option.binaryview_col LIKE Utf8("%a%") +02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8("%a%"), binary_as_string_option.largebinary_col LIKE Utf8("%a%"), binary_as_string_option.binaryview_col LIKE Utf8("%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)------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% + + +statement ok +DROP TABLE binary_as_string_option; + +## Test 3: Read table with binary_as_string option AND schema_force_view_types + +statement ok +CREATE EXTERNAL TABLE binary_as_string_both +STORED AS PARQUET LOCATION 'test_files/scratch/parquet/binary_as_string.parquet' +OPTIONS ( + 'binary_as_string' 'true', + 'schema_force_view_types' 'true' +); + +# NB the data is read and displayed a StringView +query TTTTTT +select + arrow_typeof(binary_col), binary_col, + arrow_typeof(largebinary_col), largebinary_col, + arrow_typeof(binaryview_col), binaryview_col + FROM binary_as_string_both; +---- +Utf8View aaa Utf8View aaa Utf8View aaa +Utf8View bbb Utf8View bbb Utf8View bbb +Utf8View ccc Utf8View ccc Utf8View ccc +Utf8View ddd Utf8View ddd Utf8View ddd +Utf8View eee Utf8View eee Utf8View eee +Utf8View fff Utf8View fff Utf8View fff +Utf8View ggg Utf8View ggg Utf8View ggg +Utf8View hhh Utf8View hhh Utf8View hhh +Utf8View iii Utf8View iii Utf8View iii + +# Run an explain plan to show the cast happens in the plan (there should be no casts) +query TT +EXPLAIN + SELECT binary_col, largebinary_col, binaryview_col + FROM binary_as_string_both + WHERE + binary_col LIKE '%a%' AND + largebinary_col LIKE '%a%' AND + binaryview_col LIKE '%a%'; +---- +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)------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% + + +statement ok +drop table binary_as_string_both; + +# Read a parquet file with binary data in a FixedSizeBinary column + +# by default, the data is read as binary +statement ok +CREATE EXTERNAL TABLE test_non_utf8_binary +STORED AS PARQUET LOCATION '../core/tests/data/test_binary.parquet'; + +query T? +SELECT arrow_typeof(ids), ids FROM test_non_utf8_binary LIMIT 3; +---- +FixedSizeBinary(16) 008c7196f68089ab692e4739c5fd16b5 +FixedSizeBinary(16) 00a51a7bc5ff8eb1627f8f3dc959dce8 +FixedSizeBinary(16) 0166ce1d46129ad104fa4990c6057c91 + +statement ok +DROP TABLE test_non_utf8_binary; + + +# even with the binary_as_string option set, the data is read as binary +statement ok +CREATE EXTERNAL TABLE test_non_utf8_binary +STORED AS PARQUET LOCATION '../core/tests/data/test_binary.parquet' +OPTIONS ('binary_as_string' 'true'); + +query T? +SELECT arrow_typeof(ids), ids FROM test_non_utf8_binary LIMIT 3 +---- +FixedSizeBinary(16) 008c7196f68089ab692e4739c5fd16b5 +FixedSizeBinary(16) 00a51a7bc5ff8eb1627f8f3dc959dce8 +FixedSizeBinary(16) 0166ce1d46129ad104fa4990c6057c91 + +statement ok +DROP TABLE test_non_utf8_binary; diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 10917932482c..91a2e8b4389a 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -57,6 +57,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | | datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | | datafusion.execution.parquet.schema_force_view_types | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | | datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | | datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | From 02b969381db1f3765676029aa47ddf5e54ccdf4f Mon Sep 17 00:00:00 2001 From: Jagdish Parihar Date: Fri, 25 Oct 2024 18:42:58 +0530 Subject: [PATCH 13/27] Convert `ntile` builtIn function to UDWF (#13040) * converting to ntile udwf * updated the window functions documentation file * wip: update the ntile udwf function * fix the roundtrip_logical_plan.rs * removed builtIn ntile function * fixed field name issue * fixing the return type of ntile udwf * error if UInt64 conversion fails * handling if null is found * handling if value is zero or less than zero * removed unused import * updated prost.rs file * removed dead code * fixed clippy error * added inner doc comment * minor fixes and added roundtrip logical plan test * removed parse_expr in ntile --- .../expr/src/built_in_window_function.rs | 19 -- datafusion/expr/src/expr.rs | 11 +- datafusion/expr/src/window_function.rs | 5 - datafusion/functions-window/src/lib.rs | 4 +- datafusion/functions-window/src/ntile.rs | 168 ++++++++++++++++++ datafusion/functions-window/src/utils.rs | 12 ++ .../physical-expr/src/expressions/mod.rs | 1 - datafusion/physical-expr/src/window/mod.rs | 1 - datafusion/physical-expr/src/window/ntile.rs | 111 ------------ datafusion/physical-plan/src/windows/mod.rs | 59 +----- datafusion/proto/proto/datafusion.proto | 2 +- datafusion/proto/src/generated/pbjson.rs | 3 - datafusion/proto/src/generated/prost.rs | 4 +- .../proto/src/logical_plan/from_proto.rs | 1 - datafusion/proto/src/logical_plan/to_proto.rs | 1 - .../proto/src/physical_plan/to_proto.rs | 45 ++--- .../tests/cases/roundtrip_logical_plan.rs | 3 +- .../source/user-guide/sql/window_functions.md | 34 ---- .../user-guide/sql/window_functions_new.md | 13 ++ 19 files changed, 221 insertions(+), 276 deletions(-) create mode 100644 datafusion/functions-window/src/ntile.rs delete mode 100644 datafusion/physical-expr/src/window/ntile.rs diff --git a/datafusion/expr/src/built_in_window_function.rs b/datafusion/expr/src/built_in_window_function.rs index 36916a6b594f..ab41395ad371 100644 --- a/datafusion/expr/src/built_in_window_function.rs +++ b/datafusion/expr/src/built_in_window_function.rs @@ -40,8 +40,6 @@ impl fmt::Display for BuiltInWindowFunction { /// [Window Function]: https://en.wikipedia.org/wiki/Window_function_(SQL) #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash, EnumIter)] pub enum BuiltInWindowFunction { - /// Integer ranging from 1 to the argument value, dividing the partition as equally as possible - Ntile, /// returns value evaluated at the row that is the first row of the window frame FirstValue, /// Returns value evaluated at the row that is the last row of the window frame @@ -54,7 +52,6 @@ impl BuiltInWindowFunction { pub fn name(&self) -> &str { use BuiltInWindowFunction::*; match self { - Ntile => "NTILE", FirstValue => "first_value", LastValue => "last_value", NthValue => "NTH_VALUE", @@ -66,7 +63,6 @@ impl FromStr for BuiltInWindowFunction { type Err = DataFusionError; fn from_str(name: &str) -> Result { Ok(match name.to_uppercase().as_str() { - "NTILE" => BuiltInWindowFunction::Ntile, "FIRST_VALUE" => BuiltInWindowFunction::FirstValue, "LAST_VALUE" => BuiltInWindowFunction::LastValue, "NTH_VALUE" => BuiltInWindowFunction::NthValue, @@ -97,7 +93,6 @@ impl BuiltInWindowFunction { })?; match self { - BuiltInWindowFunction::Ntile => Ok(DataType::UInt64), BuiltInWindowFunction::FirstValue | BuiltInWindowFunction::LastValue | BuiltInWindowFunction::NthValue => Ok(input_expr_types[0].clone()), @@ -111,20 +106,6 @@ impl BuiltInWindowFunction { BuiltInWindowFunction::FirstValue | BuiltInWindowFunction::LastValue => { Signature::any(1, Volatility::Immutable) } - BuiltInWindowFunction::Ntile => Signature::uniform( - 1, - vec![ - DataType::UInt64, - DataType::UInt32, - DataType::UInt16, - DataType::UInt8, - DataType::Int64, - DataType::Int32, - DataType::Int16, - DataType::Int8, - ], - Volatility::Immutable, - ), BuiltInWindowFunction::NthValue => Signature::any(2, Volatility::Immutable), } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 7fadf6391bf3..4d73c2a04486 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -2567,18 +2567,9 @@ mod test { Ok(()) } - #[test] - fn test_ntile_return_type() -> Result<()> { - let fun = find_df_window_func("ntile").unwrap(); - let observed = fun.return_type(&[DataType::Int16], &[true], "")?; - assert_eq!(DataType::UInt64, observed); - - Ok(()) - } - #[test] fn test_window_function_case_insensitive() -> Result<()> { - let names = vec!["ntile", "first_value", "last_value", "nth_value"]; + let names = vec!["first_value", "last_value", "nth_value"]; for name in names { let fun = find_df_window_func(name).unwrap(); let fun2 = find_df_window_func(name.to_uppercase().as_str()).unwrap(); diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index c13a028e4a30..be2b6575e2e9 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -17,11 +17,6 @@ use crate::{expr::WindowFunction, BuiltInWindowFunction, Expr, Literal}; -/// Create an expression to represent the `ntile` window function -pub fn ntile(arg: Expr) -> Expr { - Expr::WindowFunction(WindowFunction::new(BuiltInWindowFunction::Ntile, vec![arg])) -} - /// Create an expression to represent the `nth_value` window function pub fn nth_value(arg: Expr, n: i64) -> Expr { Expr::WindowFunction(WindowFunction::new( diff --git a/datafusion/functions-window/src/lib.rs b/datafusion/functions-window/src/lib.rs index 13a77977d579..ff8542838df9 100644 --- a/datafusion/functions-window/src/lib.rs +++ b/datafusion/functions-window/src/lib.rs @@ -34,7 +34,7 @@ pub mod macros; pub mod cume_dist; pub mod lead_lag; - +pub mod ntile; pub mod rank; pub mod row_number; mod utils; @@ -44,6 +44,7 @@ pub mod expr_fn { pub use super::cume_dist::cume_dist; pub use super::lead_lag::lag; pub use super::lead_lag::lead; + pub use super::ntile::ntile; pub use super::rank::{dense_rank, percent_rank, rank}; pub use super::row_number::row_number; } @@ -58,6 +59,7 @@ pub fn all_default_window_functions() -> Vec> { rank::rank_udwf(), rank::dense_rank_udwf(), rank::percent_rank_udwf(), + ntile::ntile_udwf(), ] } /// Registers all enabled packages with a [`FunctionRegistry`] diff --git a/datafusion/functions-window/src/ntile.rs b/datafusion/functions-window/src/ntile.rs new file mode 100644 index 000000000000..b0a7241f24cd --- /dev/null +++ b/datafusion/functions-window/src/ntile.rs @@ -0,0 +1,168 @@ +// 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. + +//! `ntile` window function implementation + +use std::any::Any; +use std::fmt::Debug; +use std::sync::{Arc, OnceLock}; + +use crate::utils::{ + get_scalar_value_from_args, get_signed_integer, get_unsigned_integer, +}; +use datafusion_common::arrow::array::{ArrayRef, UInt64Array}; +use datafusion_common::arrow::datatypes::{DataType, Field}; +use datafusion_common::{exec_err, DataFusionError, Result}; +use datafusion_expr::window_doc_sections::DOC_SECTION_RANKING; +use datafusion_expr::{ + Documentation, Expr, PartitionEvaluator, Signature, Volatility, WindowUDFImpl, +}; +use datafusion_functions_window_common::field; +use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; +use field::WindowUDFFieldArgs; + +get_or_init_udwf!( + Ntile, + ntile, + "integer ranging from 1 to the argument value, dividing the partition as equally as possible" +); + +pub fn ntile(arg: Expr) -> Expr { + ntile_udwf().call(vec![arg]) +} + +#[derive(Debug)] +pub struct Ntile { + signature: Signature, +} + +impl Ntile { + /// Create a new `ntile` function + pub fn new() -> Self { + Self { + signature: Signature::uniform( + 1, + vec![ + DataType::UInt64, + DataType::UInt32, + DataType::UInt16, + DataType::UInt8, + DataType::Int64, + DataType::Int32, + DataType::Int16, + DataType::Int8, + ], + Volatility::Immutable, + ), + } + } +} + +impl Default for Ntile { + fn default() -> Self { + Self::new() + } +} + +static DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_ntile_doc() -> &'static Documentation { + DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_RANKING) + .with_description( + "Integer ranging from 1 to the argument value, dividing the partition as equally as possible", + ) + .with_syntax_example("ntile(expression)") + .with_argument("expression","An integer describing the number groups the partition should be split into") + .build() + .unwrap() + }) +} + +impl WindowUDFImpl for Ntile { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "ntile" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn partition_evaluator( + &self, + partition_evaluator_args: PartitionEvaluatorArgs, + ) -> Result> { + let scalar_n = + get_scalar_value_from_args(partition_evaluator_args.input_exprs(), 0)? + .ok_or_else(|| { + DataFusionError::Execution( + "NTILE requires a positive integer".to_string(), + ) + })?; + + if scalar_n.is_null() { + return exec_err!("NTILE requires a positive integer, but finds NULL"); + } + + if scalar_n.is_unsigned() { + let n = get_unsigned_integer(scalar_n)?; + Ok(Box::new(NtileEvaluator { n })) + } else { + let n: i64 = get_signed_integer(scalar_n)?; + if n <= 0 { + return exec_err!("NTILE requires a positive integer"); + } + Ok(Box::new(NtileEvaluator { n: n as u64 })) + } + } + fn field(&self, field_args: WindowUDFFieldArgs) -> Result { + let nullable = false; + + Ok(Field::new(field_args.name(), DataType::UInt64, nullable)) + } + + fn documentation(&self) -> Option<&Documentation> { + Some(get_ntile_doc()) + } +} + +#[derive(Debug)] +struct NtileEvaluator { + n: u64, +} + +impl PartitionEvaluator for NtileEvaluator { + fn evaluate_all( + &mut self, + _values: &[ArrayRef], + num_rows: usize, + ) -> Result { + let num_rows = num_rows as u64; + let mut vec: Vec = Vec::new(); + let n = u64::min(self.n, num_rows); + for i in 0..num_rows { + let res = i * n / num_rows; + vec.push(res + 1) + } + Ok(Arc::new(UInt64Array::from(vec))) + } +} diff --git a/datafusion/functions-window/src/utils.rs b/datafusion/functions-window/src/utils.rs index 69f68aa78f2c..3f8061dbea3e 100644 --- a/datafusion/functions-window/src/utils.rs +++ b/datafusion/functions-window/src/utils.rs @@ -51,3 +51,15 @@ pub(crate) fn get_scalar_value_from_args( None }) } + +pub(crate) fn get_unsigned_integer(value: ScalarValue) -> Result { + if value.is_null() { + return Ok(0); + } + + if !value.data_type().is_integer() { + return exec_err!("Expected an integer value"); + } + + value.cast_to(&DataType::UInt64)?.try_into() +} diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 63047f6929c1..7d71bd9ff17b 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -36,7 +36,6 @@ mod unknown_column; /// Module with some convenient methods used in expression building pub use crate::aggregate::stats::StatsType; pub use crate::window::nth_value::NthValue; -pub use crate::window::ntile::Ntile; pub use crate::PhysicalSortExpr; pub use binary::{binary, similar_to, BinaryExpr}; diff --git a/datafusion/physical-expr/src/window/mod.rs b/datafusion/physical-expr/src/window/mod.rs index 7bab4dbc5af6..3c37fff7a1ba 100644 --- a/datafusion/physical-expr/src/window/mod.rs +++ b/datafusion/physical-expr/src/window/mod.rs @@ -19,7 +19,6 @@ mod aggregate; mod built_in; mod built_in_window_function_expr; pub(crate) mod nth_value; -pub(crate) mod ntile; mod sliding_aggregate; mod window_expr; diff --git a/datafusion/physical-expr/src/window/ntile.rs b/datafusion/physical-expr/src/window/ntile.rs deleted file mode 100644 index fb7a7ad84fb7..000000000000 --- a/datafusion/physical-expr/src/window/ntile.rs +++ /dev/null @@ -1,111 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Defines physical expression for `ntile` that can evaluated -//! at runtime during query execution - -use crate::expressions::Column; -use crate::window::BuiltInWindowFunctionExpr; -use crate::{PhysicalExpr, PhysicalSortExpr}; - -use arrow::array::{ArrayRef, UInt64Array}; -use arrow::datatypes::Field; -use arrow_schema::{DataType, SchemaRef, SortOptions}; -use datafusion_common::Result; -use datafusion_expr::PartitionEvaluator; - -use std::any::Any; -use std::sync::Arc; - -#[derive(Debug)] -pub struct Ntile { - name: String, - n: u64, - /// Output data type - data_type: DataType, -} - -impl Ntile { - pub fn new(name: String, n: u64, data_type: &DataType) -> Self { - Self { - name, - n, - data_type: data_type.clone(), - } - } - - pub fn get_n(&self) -> u64 { - self.n - } -} - -impl BuiltInWindowFunctionExpr for Ntile { - fn as_any(&self) -> &dyn Any { - self - } - - fn field(&self) -> Result { - let nullable = false; - Ok(Field::new(self.name(), self.data_type.clone(), nullable)) - } - - fn expressions(&self) -> Vec> { - vec![] - } - - fn name(&self) -> &str { - &self.name - } - - fn create_evaluator(&self) -> Result> { - Ok(Box::new(NtileEvaluator { n: self.n })) - } - - fn get_result_ordering(&self, schema: &SchemaRef) -> Option { - // The built-in NTILE window function introduces a new ordering: - schema.column_with_name(self.name()).map(|(idx, field)| { - let expr = Arc::new(Column::new(field.name(), idx)); - let options = SortOptions { - descending: false, - nulls_first: false, - }; // ASC, NULLS LAST - PhysicalSortExpr { expr, options } - }) - } -} - -#[derive(Debug)] -pub(crate) struct NtileEvaluator { - n: u64, -} - -impl PartitionEvaluator for NtileEvaluator { - fn evaluate_all( - &mut self, - _values: &[ArrayRef], - num_rows: usize, - ) -> Result { - let num_rows = num_rows as u64; - let mut vec: Vec = Vec::new(); - let n = u64::min(self.n, num_rows); - for i in 0..num_rows { - let res = i * n / num_rows; - vec.push(res + 1) - } - Ok(Arc::new(UInt64Array::from(vec))) - } -} diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 39ff71496e21..7ebb7e71ec57 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -21,15 +21,13 @@ use std::borrow::Borrow; use std::sync::Arc; use crate::{ - expressions::{Literal, NthValue, Ntile, PhysicalSortExpr}, + expressions::{Literal, NthValue, PhysicalSortExpr}, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, PhysicalExpr, }; use arrow::datatypes::Schema; use arrow_schema::{DataType, Field, SchemaRef}; -use datafusion_common::{ - exec_datafusion_err, exec_err, DataFusionError, Result, ScalarValue, -}; +use datafusion_common::{exec_datafusion_err, exec_err, Result, ScalarValue}; use datafusion_expr::{ BuiltInWindowFunction, PartitionEvaluator, ReversedUDWF, WindowFrame, WindowFunctionDefinition, WindowUDF, @@ -165,25 +163,6 @@ fn window_expr_from_aggregate_expr( } } -fn get_scalar_value_from_args( - args: &[Arc], - index: usize, -) -> Result> { - Ok(if let Some(field) = args.get(index) { - let tmp = field - .as_any() - .downcast_ref::() - .ok_or_else(|| DataFusionError::NotImplemented( - format!("There is only support Literal types for field at idx: {index} in Window Function"), - ))? - .value() - .clone(); - Some(tmp) - } else { - None - }) -} - fn get_signed_integer(value: ScalarValue) -> Result { if value.is_null() { return Ok(0); @@ -196,18 +175,6 @@ fn get_signed_integer(value: ScalarValue) -> Result { value.cast_to(&DataType::Int64)?.try_into() } -fn get_unsigned_integer(value: ScalarValue) -> Result { - if value.is_null() { - return Ok(0); - } - - if !value.data_type().is_integer() { - return exec_err!("Expected an integer value"); - } - - value.cast_to(&DataType::UInt64)?.try_into() -} - fn create_built_in_window_expr( fun: &BuiltInWindowFunction, args: &[Arc], @@ -219,28 +186,6 @@ fn create_built_in_window_expr( let out_data_type: &DataType = input_schema.field_with_name(&name)?.data_type(); Ok(match fun { - BuiltInWindowFunction::Ntile => { - let n = get_scalar_value_from_args(args, 0)?.ok_or_else(|| { - DataFusionError::Execution( - "NTILE requires a positive integer".to_string(), - ) - })?; - - if n.is_null() { - return exec_err!("NTILE requires a positive integer, but finds NULL"); - } - - if n.is_unsigned() { - let n = get_unsigned_integer(n)?; - Arc::new(Ntile::new(name, n, out_data_type)) - } else { - let n: i64 = get_signed_integer(n)?; - if n <= 0 { - return exec_err!("NTILE requires a positive integer"); - } - Arc::new(Ntile::new(name, n as u64, out_data_type)) - } - } BuiltInWindowFunction::NthValue => { let arg = Arc::clone(&args[0]); let n = get_signed_integer( diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index c92328278e83..b68c47c57eb9 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -512,7 +512,7 @@ enum BuiltInWindowFunction { // DENSE_RANK = 2; // PERCENT_RANK = 3; // CUME_DIST = 4; - NTILE = 5; + // NTILE = 5; // LAG = 6; // LEAD = 7; FIRST_VALUE = 8; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index ca331cdaa513..e54edb718808 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -1662,7 +1662,6 @@ impl serde::Serialize for BuiltInWindowFunction { { let variant = match self { Self::Unspecified => "UNSPECIFIED", - Self::Ntile => "NTILE", Self::FirstValue => "FIRST_VALUE", Self::LastValue => "LAST_VALUE", Self::NthValue => "NTH_VALUE", @@ -1678,7 +1677,6 @@ impl<'de> serde::Deserialize<'de> for BuiltInWindowFunction { { const FIELDS: &[&str] = &[ "UNSPECIFIED", - "NTILE", "FIRST_VALUE", "LAST_VALUE", "NTH_VALUE", @@ -1723,7 +1721,6 @@ impl<'de> serde::Deserialize<'de> for BuiltInWindowFunction { { match value { "UNSPECIFIED" => Ok(BuiltInWindowFunction::Unspecified), - "NTILE" => Ok(BuiltInWindowFunction::Ntile), "FIRST_VALUE" => Ok(BuiltInWindowFunction::FirstValue), "LAST_VALUE" => Ok(BuiltInWindowFunction::LastValue), "NTH_VALUE" => Ok(BuiltInWindowFunction::NthValue), diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index fb0b3bcb2c13..dfc30e809108 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1819,7 +1819,7 @@ pub enum BuiltInWindowFunction { /// DENSE_RANK = 2; /// PERCENT_RANK = 3; /// CUME_DIST = 4; - Ntile = 5, + /// NTILE = 5; /// LAG = 6; /// LEAD = 7; FirstValue = 8, @@ -1834,7 +1834,6 @@ impl BuiltInWindowFunction { pub fn as_str_name(&self) -> &'static str { match self { Self::Unspecified => "UNSPECIFIED", - Self::Ntile => "NTILE", Self::FirstValue => "FIRST_VALUE", Self::LastValue => "LAST_VALUE", Self::NthValue => "NTH_VALUE", @@ -1844,7 +1843,6 @@ impl BuiltInWindowFunction { pub fn from_str_name(value: &str) -> ::core::option::Option { match value { "UNSPECIFIED" => Some(Self::Unspecified), - "NTILE" => Some(Self::Ntile), "FIRST_VALUE" => Some(Self::FirstValue), "LAST_VALUE" => Some(Self::LastValue), "NTH_VALUE" => Some(Self::NthValue), diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 4587c090c96a..27bda7dd5ace 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -152,7 +152,6 @@ impl From for BuiltInWindowFunction { match built_in_function { protobuf::BuiltInWindowFunction::Unspecified => todo!(), protobuf::BuiltInWindowFunction::FirstValue => Self::FirstValue, - protobuf::BuiltInWindowFunction::Ntile => Self::Ntile, protobuf::BuiltInWindowFunction::NthValue => Self::NthValue, protobuf::BuiltInWindowFunction::LastValue => Self::LastValue, } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index dce0cd741fd3..5a6f3a32c668 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -127,7 +127,6 @@ impl From<&BuiltInWindowFunction> for protobuf::BuiltInWindowFunction { BuiltInWindowFunction::FirstValue => Self::FirstValue, BuiltInWindowFunction::LastValue => Self::LastValue, BuiltInWindowFunction::NthValue => Self::NthValue, - BuiltInWindowFunction::Ntile => Self::Ntile, } } } diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 37ea6a2b47be..89a2403922e9 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -24,7 +24,7 @@ use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ BinaryExpr, CaseExpr, CastExpr, Column, InListExpr, IsNotNullExpr, IsNullExpr, - Literal, NegativeExpr, NotExpr, NthValue, Ntile, TryCastExpr, + Literal, NegativeExpr, NotExpr, NthValue, TryCastExpr, }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; @@ -108,33 +108,24 @@ pub fn serialize_physical_window_expr( let expr = built_in_window_expr.get_built_in_func_expr(); let built_in_fn_expr = expr.as_any(); - let builtin_fn = if let Some(ntile_expr) = - built_in_fn_expr.downcast_ref::() - { - args.insert( - 0, - Arc::new(Literal::new(datafusion_common::ScalarValue::Int64(Some( - ntile_expr.get_n() as i64, - )))), - ); - protobuf::BuiltInWindowFunction::Ntile - } else if let Some(nth_value_expr) = built_in_fn_expr.downcast_ref::() { - match nth_value_expr.get_kind() { - NthValueKind::First => protobuf::BuiltInWindowFunction::FirstValue, - NthValueKind::Last => protobuf::BuiltInWindowFunction::LastValue, - NthValueKind::Nth(n) => { - args.insert( - 1, - Arc::new(Literal::new(datafusion_common::ScalarValue::Int64( - Some(n), - ))), - ); - protobuf::BuiltInWindowFunction::NthValue + let builtin_fn = + if let Some(nth_value_expr) = built_in_fn_expr.downcast_ref::() { + match nth_value_expr.get_kind() { + NthValueKind::First => protobuf::BuiltInWindowFunction::FirstValue, + NthValueKind::Last => protobuf::BuiltInWindowFunction::LastValue, + NthValueKind::Nth(n) => { + args.insert( + 1, + Arc::new(Literal::new( + datafusion_common::ScalarValue::Int64(Some(n)), + )), + ); + protobuf::BuiltInWindowFunction::NthValue + } } - } - } else { - return not_impl_err!("BuiltIn function not supported: {expr:?}"); - }; + } else { + return not_impl_err!("BuiltIn function not supported: {expr:?}"); + }; ( physical_window_expr_node::WindowFunction::BuiltInFunction(builtin_fn as i32), diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index a8c82ff80f23..3fec7d1c6ea0 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -48,7 +48,7 @@ use datafusion::functions_aggregate::expr_fn::{ use datafusion::functions_aggregate::min_max::max_udaf; use datafusion::functions_nested::map::map; use datafusion::functions_window::expr_fn::{ - cume_dist, dense_rank, lag, lead, percent_rank, rank, row_number, + cume_dist, dense_rank, lag, lead, ntile, percent_rank, rank, row_number, }; use datafusion::functions_window::rank::rank_udwf; use datafusion::prelude::*; @@ -951,6 +951,7 @@ async fn roundtrip_expr_api() -> Result<()> { lag(col("b"), None, None), lag(col("b"), Some(2), None), lag(col("b"), Some(2), Some(ScalarValue::from(100))), + ntile(lit(3)), nth_value(col("b"), 1, vec![]), nth_value( col("b"), diff --git a/docs/source/user-guide/sql/window_functions.md b/docs/source/user-guide/sql/window_functions.md index 0799859e4371..6bf2005dabf9 100644 --- a/docs/source/user-guide/sql/window_functions.md +++ b/docs/source/user-guide/sql/window_functions.md @@ -146,40 +146,6 @@ RANGE and GROUPS modes require an ORDER BY clause (with RANGE the ORDER BY must All [aggregate functions](aggregate_functions.md) can be used as window functions. -## Ranking functions - -- [rank](#rank) -- [dense_rank](#dense_rank) -- [ntile](#ntile) - -### `rank` - -Rank of the current row with gaps; same as row_number of its first peer. - -```sql -rank() -``` - -### `dense_rank` - -Rank of the current row without gaps; this function counts peer groups. - -```sql -dense_rank() -``` - -### `ntile` - -Integer ranging from 1 to the argument value, dividing the partition as equally as possible. - -```sql -ntile(expression) -``` - -#### Arguments - -- **expression**: An integer describing the number groups the partition should be split into - ## Analytical functions - [cume_dist](#cume_dist) diff --git a/docs/source/user-guide/sql/window_functions_new.md b/docs/source/user-guide/sql/window_functions_new.md index 267060abfdcc..ae3edb832fcb 100644 --- a/docs/source/user-guide/sql/window_functions_new.md +++ b/docs/source/user-guide/sql/window_functions_new.md @@ -159,6 +159,7 @@ All [aggregate functions](aggregate_functions.md) can be used as window function - [cume_dist](#cume_dist) - [dense_rank](#dense_rank) +- [ntile](#ntile) - [percent_rank](#percent_rank) - [rank](#rank) - [row_number](#row_number) @@ -179,6 +180,18 @@ Returns the rank of the current row without gaps. This function ranks rows in a dense_rank() ``` +### `ntile` + +Integer ranging from 1 to the argument value, dividing the partition as equally as possible + +``` +ntile(expression) +``` + +#### Arguments + +- **expression**: An integer describing the number groups the partition should be split into + ### `percent_rank` Returns the percentage rank of the current row within its partition. The value ranges from 0 to 1 and is computed as `(rank - 1) / (total_rows - 1)`. From 7f32dcef3349059c7ee9ae0f24a3373ee2473982 Mon Sep 17 00:00:00 2001 From: June <61218022+itsjunetime@users.noreply.github.com> Date: Fri, 25 Oct 2024 07:18:55 -0600 Subject: [PATCH 14/27] Fix more instances of schema missing metadata (#13068) --- .../core/src/datasource/file_format/parquet.rs | 3 ++- datafusion/core/src/datasource/listing/table.rs | 8 +++++++- .../datasource/physical_plan/file_scan_config.rs | 14 ++++++++++---- datafusion/core/src/datasource/schema_adapter.rs | 3 ++- datafusion/core/src/physical_planner.rs | 11 +++++++++-- datafusion/expr/src/logical_plan/builder.rs | 8 ++++++-- datafusion/expr/src/utils.rs | 5 ++++- datafusion/physical-plan/src/joins/utils.rs | 8 +++++++- datafusion/physical-plan/src/windows/utils.rs | 4 +++- 9 files changed, 50 insertions(+), 14 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 756c17fd67c6..2d45c76ce918 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -738,13 +738,14 @@ impl ParquetSink { .iter() .map(|(s, _)| s) .collect(); - Arc::new(Schema::new( + Arc::new(Schema::new_with_metadata( schema .fields() .iter() .filter(|f| !partition_names.contains(&f.name())) .map(|f| (**f).clone()) .collect::>(), + schema.metadata().clone(), )) } else { self.config.output_schema().clone() diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 1e9f06c20b47..ea2e098ef14e 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -719,10 +719,16 @@ impl ListingTable { builder.push(Field::new(part_col_name, part_col_type.clone(), false)); } + let table_schema = Arc::new( + builder + .finish() + .with_metadata(file_schema.metadata().clone()), + ); + let table = Self { table_paths: config.table_paths, file_schema, - table_schema: Arc::new(builder.finish()), + table_schema, options, definition: None, collected_statistics: Arc::new(DefaultFileStatisticsCache::default()), diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 2c438e8b0e78..415ea62b3bb3 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -248,9 +248,10 @@ impl FileScanConfig { column_statistics: table_cols_stats, }; - let projected_schema = Arc::new( - Schema::new(table_fields).with_metadata(self.file_schema.metadata().clone()), - ); + let projected_schema = Arc::new(Schema::new_with_metadata( + table_fields, + self.file_schema.metadata().clone(), + )); let projected_output_ordering = get_projected_output_ordering(self, &projected_schema); @@ -281,7 +282,12 @@ impl FileScanConfig { fields.map_or_else( || Arc::clone(&self.file_schema), - |f| Arc::new(Schema::new(f).with_metadata(self.file_schema.metadata.clone())), + |f| { + Arc::new(Schema::new_with_metadata( + f, + self.file_schema.metadata.clone(), + )) + }, ) } diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index fdf3381758a4..131b8c354ce7 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -304,7 +304,8 @@ impl SchemaMapper for SchemaMapping { // Necessary to handle empty batches let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); - let schema = Arc::new(Schema::new(fields)); + let schema = + Arc::new(Schema::new_with_metadata(fields, schema.metadata().clone())); let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?; Ok(record_batch) } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 5a4ae868d04a..ffedc2d6b6ef 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1025,14 +1025,21 @@ impl DefaultPhysicalPlanner { }) .collect(); + let metadata: HashMap<_, _> = left_df_schema + .metadata() + .clone() + .into_iter() + .chain(right_df_schema.metadata().clone()) + .collect(); + // Construct intermediate schemas used for filtering data and // convert logical expression to physical according to filter schema let filter_df_schema = DFSchema::new_with_metadata( filter_df_fields, - HashMap::new(), + metadata.clone(), )?; let filter_schema = - Schema::new_with_metadata(filter_fields, HashMap::new()); + Schema::new_with_metadata(filter_fields, metadata); let filter_expr = create_physical_expr( expr, &filter_df_schema, diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index cef05b6f8814..aef531a9dbf7 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1402,8 +1402,12 @@ pub fn build_join_schema( join_type, left.fields().len(), ); - let mut metadata = left.metadata().clone(); - metadata.extend(right.metadata().clone()); + let metadata = left + .metadata() + .clone() + .into_iter() + .chain(right.metadata().clone()) + .collect(); let dfschema = DFSchema::new_with_metadata(qualified_fields, metadata)?; dfschema.with_functional_dependencies(func_dependencies) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 86562daf6909..bb5496c0f799 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -437,7 +437,10 @@ pub fn expand_qualified_wildcard( return plan_err!("Invalid qualifier {qualifier}"); } - let qualified_schema = Arc::new(Schema::new(fields_with_qualified)); + let qualified_schema = Arc::new(Schema::new_with_metadata( + fields_with_qualified, + schema.metadata().clone(), + )); let qualified_dfschema = DFSchema::try_from_qualified_schema(qualifier.clone(), &qualified_schema)? .with_functional_dependencies(projected_func_dependencies)?; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index c520e4271416..17a32a67c743 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -701,7 +701,13 @@ pub fn build_join_schema( .unzip(), }; - (fields.finish(), column_indices) + let metadata = left + .metadata() + .clone() + .into_iter() + .chain(right.metadata().clone()) + .collect(); + (fields.finish().with_metadata(metadata), column_indices) } /// A [`OnceAsync`] can be used to run an async closure once, with subsequent calls diff --git a/datafusion/physical-plan/src/windows/utils.rs b/datafusion/physical-plan/src/windows/utils.rs index 3cf92daae0fb..13332ea82fa1 100644 --- a/datafusion/physical-plan/src/windows/utils.rs +++ b/datafusion/physical-plan/src/windows/utils.rs @@ -31,5 +31,7 @@ pub(crate) fn create_schema( for expr in window_expr { builder.push(expr.field()?); } - Ok(builder.finish()) + Ok(builder + .finish() + .with_metadata(input_schema.metadata().clone())) } From 06594c79f11d57b88467e9b87e5cb1ac6cf85d88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Fri, 25 Oct 2024 16:50:38 +0300 Subject: [PATCH 15/27] Bug-fix / Limit with_new_exprs() (#13109) * Update plan.rs * Update plan.rs * Update plan.rs --- datafusion/expr/src/logical_plan/plan.rs | 26 +++++++++++++++++++++--- 1 file changed, 23 insertions(+), 3 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 4b42702f24bf..572285defba0 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -936,9 +936,8 @@ impl LogicalPlan { expr.len() ); } - // Pop order is same as the order returned by `LogicalPlan::expressions()` - let new_skip = skip.as_ref().and(expr.pop()); - let new_fetch = fetch.as_ref().and(expr.pop()); + let new_skip = skip.as_ref().and_then(|_| expr.pop()); + let new_fetch = fetch.as_ref().and_then(|_| expr.pop()); let input = self.only_input(inputs)?; Ok(LogicalPlan::Limit(Limit { skip: new_skip.map(Box::new), @@ -4101,4 +4100,25 @@ digraph { ); assert_eq!(describe_table.partial_cmp(&describe_table_clone), None); } + + #[test] + fn test_limit_with_new_children() { + let limit = LogicalPlan::Limit(Limit { + skip: None, + fetch: Some(Box::new(Expr::Literal( + ScalarValue::new_ten(&DataType::UInt32).unwrap(), + ))), + input: Arc::new(LogicalPlan::Values(Values { + schema: Arc::new(DFSchema::empty()), + values: vec![vec![]], + })), + }); + let new_limit = limit + .with_new_exprs( + limit.expressions(), + limit.inputs().into_iter().cloned().collect(), + ) + .unwrap(); + assert_eq!(limit, new_limit); + } } From 813220d54f08c5203ad79bfb066ca638abe208ed Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Fri, 25 Oct 2024 16:21:05 +0200 Subject: [PATCH 16/27] Move subquery check from analyzer to PullUpCorrelatedExpr (#13091) This patch moves subquery check `can_pull_over_aggregation` from analyzer into the PullUpCorrelatedExpr. Instead of failing the query we will no instead not decorrelate such queries and then fail during physical plan creation. The goal here is to support TPC-DS q41 which has an expression that can not be pull up until it has been simplified by SimplifyExpressions. This means that currently we reject the query already in the analyzer. But after this change we are able to plan that query. --- datafusion/core/benches/sql_planner.rs | 4 - datafusion/core/tests/tpcds_planning.rs | 5 - datafusion/optimizer/src/analyzer/subquery.rs | 96 ++++--------------- datafusion/optimizer/src/decorrelate.rs | 45 ++++++++- .../optimizer/src/scalar_subquery_to_join.rs | 54 ++++++++--- .../sqllogictest/test_files/subquery.slt | 14 ++- 6 files changed, 116 insertions(+), 102 deletions(-) diff --git a/datafusion/core/benches/sql_planner.rs b/datafusion/core/benches/sql_planner.rs index 64d2760e9d97..6f9cf02873d1 100644 --- a/datafusion/core/benches/sql_planner.rs +++ b/datafusion/core/benches/sql_planner.rs @@ -270,11 +270,7 @@ fn criterion_benchmark(c: &mut Criterion) { let tpcds_ctx = register_defs(SessionContext::new(), tpcds_schemas()); - // 41: check_analyzed_plan: Correlated column is not allowed in predicate - let ignored = [41]; - let raw_tpcds_sql_queries = (1..100) - .filter(|q| !ignored.contains(q)) .map(|q| std::fs::read_to_string(format!("./tests/tpc-ds/{q}.sql")).unwrap()) .collect::>(); diff --git a/datafusion/core/tests/tpcds_planning.rs b/datafusion/core/tests/tpcds_planning.rs index 0077a2d35b1f..252d76d0f9d9 100644 --- a/datafusion/core/tests/tpcds_planning.rs +++ b/datafusion/core/tests/tpcds_planning.rs @@ -229,9 +229,6 @@ async fn tpcds_logical_q40() -> Result<()> { } #[tokio::test] -#[ignore] -// check_analyzed_plan: Correlated column is not allowed in predicate -// issue: https://github.com/apache/datafusion/issues/13074 async fn tpcds_logical_q41() -> Result<()> { create_logical_plan(41).await } @@ -726,8 +723,6 @@ async fn tpcds_physical_q40() -> Result<()> { create_physical_plan(40).await } -#[ignore] -// Context("check_analyzed_plan", Plan("Correlated column is not allowed in predicate: (..) #[tokio::test] async fn tpcds_physical_q41() -> Result<()> { create_physical_plan(41).await diff --git a/datafusion/optimizer/src/analyzer/subquery.rs b/datafusion/optimizer/src/analyzer/subquery.rs index 0a52685bd681..e01ae625ed9c 100644 --- a/datafusion/optimizer/src/analyzer/subquery.rs +++ b/datafusion/optimizer/src/analyzer/subquery.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::ops::Deref; - use crate::analyzer::check_plan; use crate::utils::collect_subquery_cols; @@ -24,10 +22,7 @@ use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion}; use datafusion_common::{plan_err, Result}; use datafusion_expr::expr_rewriter::strip_outer_reference; use datafusion_expr::utils::split_conjunction; -use datafusion_expr::{ - Aggregate, BinaryExpr, Cast, Expr, Filter, Join, JoinType, LogicalPlan, Operator, - Window, -}; +use datafusion_expr::{Aggregate, Expr, Filter, Join, JoinType, LogicalPlan, Window}; /// Do necessary check on subquery expressions and fail the invalid plan /// 1) Check whether the outer plan is in the allowed outer plans list to use subquery expressions, @@ -98,7 +93,7 @@ pub fn check_subquery_expr( ) }?; } - check_correlations_in_subquery(inner_plan, true) + check_correlations_in_subquery(inner_plan) } else { if let Expr::InSubquery(subquery) = expr { // InSubquery should only return one column @@ -121,25 +116,17 @@ pub fn check_subquery_expr( Projection, Filter, Window functions, Aggregate and Join plan nodes" ), }?; - check_correlations_in_subquery(inner_plan, false) + check_correlations_in_subquery(inner_plan) } } // Recursively check the unsupported outer references in the sub query plan. -fn check_correlations_in_subquery( - inner_plan: &LogicalPlan, - is_scalar: bool, -) -> Result<()> { - check_inner_plan(inner_plan, is_scalar, false, true) +fn check_correlations_in_subquery(inner_plan: &LogicalPlan) -> Result<()> { + check_inner_plan(inner_plan, true) } // Recursively check the unsupported outer references in the sub query plan. -fn check_inner_plan( - inner_plan: &LogicalPlan, - is_scalar: bool, - is_aggregate: bool, - can_contain_outer_ref: bool, -) -> Result<()> { +fn check_inner_plan(inner_plan: &LogicalPlan, can_contain_outer_ref: bool) -> Result<()> { if !can_contain_outer_ref && inner_plan.contains_outer_reference() { return plan_err!("Accessing outer reference columns is not allowed in the plan"); } @@ -147,32 +134,18 @@ fn check_inner_plan( match inner_plan { LogicalPlan::Aggregate(_) => { inner_plan.apply_children(|plan| { - check_inner_plan(plan, is_scalar, true, can_contain_outer_ref)?; + check_inner_plan(plan, can_contain_outer_ref)?; Ok(TreeNodeRecursion::Continue) })?; Ok(()) } - LogicalPlan::Filter(Filter { - predicate, input, .. - }) => { - let (correlated, _): (Vec<_>, Vec<_>) = split_conjunction(predicate) - .into_iter() - .partition(|e| e.contains_outer()); - let maybe_unsupported = correlated - .into_iter() - .filter(|expr| !can_pullup_over_aggregation(expr)) - .collect::>(); - if is_aggregate && is_scalar && !maybe_unsupported.is_empty() { - return plan_err!( - "Correlated column is not allowed in predicate: {predicate}" - ); - } - check_inner_plan(input, is_scalar, is_aggregate, can_contain_outer_ref) + LogicalPlan::Filter(Filter { input, .. }) => { + check_inner_plan(input, can_contain_outer_ref) } LogicalPlan::Window(window) => { check_mixed_out_refer_in_window(window)?; inner_plan.apply_children(|plan| { - check_inner_plan(plan, is_scalar, is_aggregate, can_contain_outer_ref)?; + check_inner_plan(plan, can_contain_outer_ref)?; Ok(TreeNodeRecursion::Continue) })?; Ok(()) @@ -188,7 +161,7 @@ fn check_inner_plan( | LogicalPlan::Subquery(_) | LogicalPlan::SubqueryAlias(_) => { inner_plan.apply_children(|plan| { - check_inner_plan(plan, is_scalar, is_aggregate, can_contain_outer_ref)?; + check_inner_plan(plan, can_contain_outer_ref)?; Ok(TreeNodeRecursion::Continue) })?; Ok(()) @@ -201,27 +174,22 @@ fn check_inner_plan( }) => match join_type { JoinType::Inner => { inner_plan.apply_children(|plan| { - check_inner_plan( - plan, - is_scalar, - is_aggregate, - can_contain_outer_ref, - )?; + check_inner_plan(plan, can_contain_outer_ref)?; Ok(TreeNodeRecursion::Continue) })?; Ok(()) } JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - check_inner_plan(left, is_scalar, is_aggregate, can_contain_outer_ref)?; - check_inner_plan(right, is_scalar, is_aggregate, false) + check_inner_plan(left, can_contain_outer_ref)?; + check_inner_plan(right, false) } JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { - check_inner_plan(left, is_scalar, is_aggregate, false)?; - check_inner_plan(right, is_scalar, is_aggregate, can_contain_outer_ref) + check_inner_plan(left, false)?; + check_inner_plan(right, can_contain_outer_ref) } JoinType::Full => { inner_plan.apply_children(|plan| { - check_inner_plan(plan, is_scalar, is_aggregate, false)?; + check_inner_plan(plan, false)?; Ok(TreeNodeRecursion::Continue) })?; Ok(()) @@ -290,34 +258,6 @@ fn get_correlated_expressions(inner_plan: &LogicalPlan) -> Result> { Ok(exprs) } -/// Check whether the expression can pull up over the aggregation without change the result of the query -fn can_pullup_over_aggregation(expr: &Expr) -> bool { - if let Expr::BinaryExpr(BinaryExpr { - left, - op: Operator::Eq, - right, - }) = expr - { - match (left.deref(), right.deref()) { - (Expr::Column(_), right) => !right.any_column_refs(), - (left, Expr::Column(_)) => !left.any_column_refs(), - (Expr::Cast(Cast { expr, .. }), right) - if matches!(expr.deref(), Expr::Column(_)) => - { - !right.any_column_refs() - } - (left, Expr::Cast(Cast { expr, .. })) - if matches!(expr.deref(), Expr::Column(_)) => - { - !left.any_column_refs() - } - (_, _) => false, - } - } else { - false - } -} - /// Check whether the window expressions contain a mixture of out reference columns and inner columns fn check_mixed_out_refer_in_window(window: &Window) -> Result<()> { let mixed = window @@ -398,6 +338,6 @@ mod test { }), }); - check_inner_plan(&plan, false, false, true).unwrap(); + check_inner_plan(&plan, true).unwrap(); } } diff --git a/datafusion/optimizer/src/decorrelate.rs b/datafusion/optimizer/src/decorrelate.rs index baf449a045eb..6aa59b77f7f9 100644 --- a/datafusion/optimizer/src/decorrelate.rs +++ b/datafusion/optimizer/src/decorrelate.rs @@ -32,7 +32,8 @@ use datafusion_expr::expr::Alias; use datafusion_expr::simplify::SimplifyContext; use datafusion_expr::utils::{conjunction, find_join_exprs, split_conjunction}; use datafusion_expr::{ - expr, lit, EmptyRelation, Expr, FetchType, LogicalPlan, LogicalPlanBuilder, + expr, lit, BinaryExpr, Cast, EmptyRelation, Expr, FetchType, LogicalPlan, + LogicalPlanBuilder, Operator, }; use datafusion_physical_expr::execution_props::ExecutionProps; @@ -51,6 +52,9 @@ pub struct PullUpCorrelatedExpr { pub exists_sub_query: bool, /// Can the correlated expressions be pulled up. Defaults to **TRUE** pub can_pull_up: bool, + /// Indicates if we encounter any correlated expression that can not be pulled up + /// above a aggregation without changing the meaning of the query. + can_pull_over_aggregation: bool, /// Do we need to handle [the Count bug] during the pull up process /// /// [the Count bug]: https://github.com/apache/datafusion/pull/10500 @@ -75,6 +79,7 @@ impl PullUpCorrelatedExpr { in_predicate_opt: None, exists_sub_query: false, can_pull_up: true, + can_pull_over_aggregation: true, need_handle_count_bug: false, collected_count_expr_map: HashMap::new(), pull_up_having_expr: None, @@ -154,6 +159,11 @@ impl TreeNodeRewriter for PullUpCorrelatedExpr { match &plan { LogicalPlan::Filter(plan_filter) => { let subquery_filter_exprs = split_conjunction(&plan_filter.predicate); + self.can_pull_over_aggregation = self.can_pull_over_aggregation + && subquery_filter_exprs + .iter() + .filter(|e| e.contains_outer()) + .all(|&e| can_pullup_over_aggregation(e)); let (mut join_filters, subquery_filters) = find_join_exprs(subquery_filter_exprs)?; if let Some(in_predicate) = &self.in_predicate_opt { @@ -259,6 +269,12 @@ impl TreeNodeRewriter for PullUpCorrelatedExpr { LogicalPlan::Aggregate(aggregate) if self.in_predicate_opt.is_some() || !self.join_filters.is_empty() => { + // If the aggregation is from a distinct it will not change the result for + // exists/in subqueries so we can still pull up all predicates. + let is_distinct = aggregate.aggr_expr.is_empty(); + if !is_distinct { + self.can_pull_up = self.can_pull_up && self.can_pull_over_aggregation; + } let mut local_correlated_cols = BTreeSet::new(); collect_local_correlated_cols( &plan, @@ -385,6 +401,33 @@ impl PullUpCorrelatedExpr { } } +fn can_pullup_over_aggregation(expr: &Expr) -> bool { + if let Expr::BinaryExpr(BinaryExpr { + left, + op: Operator::Eq, + right, + }) = expr + { + match (left.deref(), right.deref()) { + (Expr::Column(_), right) => !right.any_column_refs(), + (left, Expr::Column(_)) => !left.any_column_refs(), + (Expr::Cast(Cast { expr, .. }), right) + if matches!(expr.deref(), Expr::Column(_)) => + { + !right.any_column_refs() + } + (left, Expr::Cast(Cast { expr, .. })) + if matches!(expr.deref(), Expr::Column(_)) => + { + !left.any_column_refs() + } + (_, _) => false, + } + } else { + false + } +} + fn collect_local_correlated_cols( plan: &LogicalPlan, all_cols_map: &HashMap>, diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index 6409bb9e03f7..7b931e73abf9 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -625,11 +625,21 @@ mod tests { .project(vec![col("customer.c_custkey")])? .build()?; - let expected = "check_analyzed_plan\ - \ncaused by\ - \nError during planning: Correlated column is not allowed in predicate: outer_ref(customer.c_custkey) != orders.o_custkey"; + // Unsupported predicate, subquery should not be decorrelated + let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ + \n Filter: customer.c_custkey = () [c_custkey:Int64, c_name:Utf8]\ + \n Subquery: [max(orders.o_custkey):Int64;N]\ + \n Projection: max(orders.o_custkey) [max(orders.o_custkey):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[max(orders.o_custkey)]] [max(orders.o_custkey):Int64;N]\ + \n Filter: outer_ref(customer.c_custkey) != orders.o_custkey [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ + \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ + \n TableScan: customer [c_custkey:Int64, c_name:Utf8]"; - assert_analyzer_check_err(vec![], plan, expected); + assert_multi_rules_optimized_plan_eq_display_indent( + vec![Arc::new(ScalarSubqueryToJoin::new())], + plan, + expected, + ); Ok(()) } @@ -652,11 +662,21 @@ mod tests { .project(vec![col("customer.c_custkey")])? .build()?; - let expected = "check_analyzed_plan\ - \ncaused by\ - \nError during planning: Correlated column is not allowed in predicate: outer_ref(customer.c_custkey) < orders.o_custkey"; + // Unsupported predicate, subquery should not be decorrelated + let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ + \n Filter: customer.c_custkey = () [c_custkey:Int64, c_name:Utf8]\ + \n Subquery: [max(orders.o_custkey):Int64;N]\ + \n Projection: max(orders.o_custkey) [max(orders.o_custkey):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[max(orders.o_custkey)]] [max(orders.o_custkey):Int64;N]\ + \n Filter: outer_ref(customer.c_custkey) < orders.o_custkey [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ + \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ + \n TableScan: customer [c_custkey:Int64, c_name:Utf8]"; - assert_analyzer_check_err(vec![], plan, expected); + assert_multi_rules_optimized_plan_eq_display_indent( + vec![Arc::new(ScalarSubqueryToJoin::new())], + plan, + expected, + ); Ok(()) } @@ -680,11 +700,21 @@ mod tests { .project(vec![col("customer.c_custkey")])? .build()?; - let expected = "check_analyzed_plan\ - \ncaused by\ - \nError during planning: Correlated column is not allowed in predicate: outer_ref(customer.c_custkey) = orders.o_custkey OR orders.o_orderkey = Int32(1)"; + // Unsupported predicate, subquery should not be decorrelated + let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ + \n Filter: customer.c_custkey = () [c_custkey:Int64, c_name:Utf8]\ + \n Subquery: [max(orders.o_custkey):Int64;N]\ + \n Projection: max(orders.o_custkey) [max(orders.o_custkey):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[max(orders.o_custkey)]] [max(orders.o_custkey):Int64;N]\ + \n Filter: outer_ref(customer.c_custkey) = orders.o_custkey OR orders.o_orderkey = Int32(1) [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ + \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ + \n TableScan: customer [c_custkey:Int64, c_name:Utf8]"; - assert_analyzer_check_err(vec![], plan, expected); + assert_multi_rules_optimized_plan_eq_display_indent( + vec![Arc::new(ScalarSubqueryToJoin::new())], + plan, + expected, + ); Ok(()) } diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 6b142302a543..26b5d8b952f6 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -509,8 +509,18 @@ SELECT t1_id, (SELECT a FROM (select 1 as a) WHERE a = t1.t1_int) as t2_int from 44 NULL #non_equal_correlated_scalar_subquery -statement error DataFusion error: check_analyzed_plan\ncaused by\nError during planning: Correlated column is not allowed in predicate: t2\.t2_id < outer_ref\(t1\.t1_id\) -SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id < t1.t1_id) as t2_sum from t1 +# Currently not supported and should not be decorrelated +query TT +explain SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id < t1.t1_id) as t2_sum from t1 +---- +logical_plan +01)Projection: t1.t1_id, () AS t2_sum +02)--Subquery: +03)----Projection: sum(t2.t2_int) +04)------Aggregate: groupBy=[[]], aggr=[[sum(CAST(t2.t2_int AS Int64))]] +05)--------Filter: t2.t2_id < outer_ref(t1.t1_id) +06)----------TableScan: t2 +07)--TableScan: t1 projection=[t1_id] #aggregated_correlated_scalar_subquery_with_extra_group_by_columns statement error DataFusion error: check_analyzed_plan\ncaused by\nError during planning: A GROUP BY clause in a scalar correlated subquery cannot contain non-correlated columns From bdcf8225933c852e9f3a1b44a51d262627506f98 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Sat, 26 Oct 2024 04:59:41 +0800 Subject: [PATCH 17/27] Include IMDB in benchmark README (#13107) --- benchmarks/README.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/benchmarks/README.md b/benchmarks/README.md index afaf28bb7576..a12662ccb846 100644 --- a/benchmarks/README.md +++ b/benchmarks/README.md @@ -330,6 +330,16 @@ steps. The tests sort the entire dataset using several different sort orders. +## IMDB + +Run Join Order Benchmark (JOB) on IMDB dataset. + +The Internet Movie Database (IMDB) dataset contains real-world movie data. Unlike synthetic datasets like TPCH, which assume uniform data distribution and uncorrelated columns, the IMDB dataset includes skewed data and correlated columns (which are common for real dataset), making it more suitable for testing query optimizers, particularly for cardinality estimation. + +This benchmark is derived from [Join Order Benchmark](https://github.com/gregrahn/join-order-benchmark). + +See paper [How Good Are Query Optimizers, Really](http://www.vldb.org/pvldb/vol9/p204-leis.pdf) for more details. + ## TPCH Run the tpch benchmark. From 96236908877d9cebbf115cff001f35f0729fab9e Mon Sep 17 00:00:00 2001 From: neyama Date: Sat, 26 Oct 2024 06:02:27 +0900 Subject: [PATCH 18/27] removed --prefer_hash_join option that causes an error when running the benchmark (#13106) --- benchmarks/bench.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 70faa9ef2b73..fc10cc5afc53 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -357,7 +357,7 @@ run_parquet() { RESULTS_FILE="${RESULTS_DIR}/parquet.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running parquet filter benchmark..." - $CARGO_COMMAND --bin parquet -- filter --path "${DATA_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin parquet -- filter --path "${DATA_DIR}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" } # Runs the sort benchmark @@ -365,7 +365,7 @@ run_sort() { RESULTS_FILE="${RESULTS_DIR}/sort.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running sort benchmark..." - $CARGO_COMMAND --bin parquet -- sort --path "${DATA_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin parquet -- sort --path "${DATA_DIR}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" } From 21cfd6ccdbd6996be24586568bc9c260f50505db Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 25 Oct 2024 17:14:06 -0400 Subject: [PATCH 19/27] Make CI test error if a function is not documented (#12938) --- .../core/src/bin/print_functions_docs.rs | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/datafusion/core/src/bin/print_functions_docs.rs b/datafusion/core/src/bin/print_functions_docs.rs index d87c3cefe666..598574c0703d 100644 --- a/datafusion/core/src/bin/print_functions_docs.rs +++ b/datafusion/core/src/bin/print_functions_docs.rs @@ -16,6 +16,7 @@ // under the License. use datafusion::execution::SessionStateDefaults; +use datafusion_common::{not_impl_err, Result}; use datafusion_expr::{ aggregate_doc_sections, scalar_doc_sections, window_doc_sections, AggregateUDF, DocSection, Documentation, ScalarUDF, WindowUDF, @@ -30,7 +31,7 @@ use std::fmt::Write as _; /// Usage: `cargo run --bin print_functions_docs -- ` /// /// Called from `dev/update_function_docs.sh` -fn main() { +fn main() -> Result<()> { let args: Vec = args().collect(); if args.len() != 2 { @@ -48,12 +49,13 @@ fn main() { _ => { panic!("Unknown function type: {}", function_type) } - }; + }?; println!("{docs}"); + Ok(()) } -fn print_aggregate_docs() -> String { +fn print_aggregate_docs() -> Result { let mut providers: Vec> = vec![]; for f in SessionStateDefaults::default_aggregate_functions() { @@ -63,7 +65,7 @@ fn print_aggregate_docs() -> String { print_docs(providers, aggregate_doc_sections::doc_sections()) } -fn print_scalar_docs() -> String { +fn print_scalar_docs() -> Result { let mut providers: Vec> = vec![]; for f in SessionStateDefaults::default_scalar_functions() { @@ -73,7 +75,7 @@ fn print_scalar_docs() -> String { print_docs(providers, scalar_doc_sections::doc_sections()) } -fn print_window_docs() -> String { +fn print_window_docs() -> Result { let mut providers: Vec> = vec![]; for f in SessionStateDefaults::default_window_functions() { @@ -86,7 +88,7 @@ fn print_window_docs() -> String { fn print_docs( providers: Vec>, doc_sections: Vec, -) -> String { +) -> Result { let mut docs = "".to_string(); // Ensure that all providers have documentation @@ -217,12 +219,13 @@ fn print_docs( // eventually make this an error: https://github.com/apache/datafusion/issues/12872 if !providers_with_no_docs.is_empty() { eprintln!("INFO: The following functions do not have documentation:"); - for f in providers_with_no_docs { + for f in &providers_with_no_docs { eprintln!(" - {f}"); } + not_impl_err!("Some functions do not have documentation. Please implement `documentation` for: {providers_with_no_docs:?}") + } else { + Ok(docs) } - - docs } /// Trait for accessing name / aliases / documentation for differnet functions From 7b2284c8a0b49234e9607bfef10d73ef788d9458 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 25 Oct 2024 17:27:23 -0400 Subject: [PATCH 20/27] Allow using `cargo nextest` for running tests (#13045) * Allow using `cargo nextest` for running tests * Update datafusion/sqllogictest/bin/sqllogictests.rs Co-authored-by: Piotr Findeisen * Clarify rationale for returning OK * Apply suggestions from code review Co-authored-by: Piotr Findeisen --------- Co-authored-by: Piotr Findeisen --- datafusion/sqllogictest/bin/sqllogictests.rs | 29 +++++++++++++++++--- 1 file changed, 25 insertions(+), 4 deletions(-) diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index baa49057e1b9..501fd3517a17 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -62,6 +62,15 @@ async fn run_tests() -> Result<()> { env_logger::init(); let options: Options = clap::Parser::parse(); + if options.list { + // nextest parses stdout, so print messages to stderr + eprintln!("NOTICE: --list option unsupported, quitting"); + // return Ok, not error so that tools like nextest which are listing all + // workspace tests (by running `cargo test ... --list --format terse`) + // do not fail when they encounter this binary. Instead, print nothing + // to stdout and return OK so they can continue listing other tests. + return Ok(()); + } options.warn_on_ignored(); // Run all tests in parallel, reporting failures at the end @@ -276,7 +285,7 @@ fn read_dir_recursive_impl(dst: &mut Vec, path: &Path) -> Result<()> { /// Parsed command line options /// -/// This structure attempts to mimic the command line options +/// This structure attempts to mimic the command line options of the built in rust test runner /// accepted by IDEs such as CLion that pass arguments /// /// See for more details @@ -320,6 +329,18 @@ struct Options { help = "IGNORED (for compatibility with built in rust test runner)" )] show_output: bool, + + #[clap( + long, + help = "Quits immediately, not listing anything (for compatibility with built-in rust test runner)" + )] + list: bool, + + #[clap( + long, + help = "IGNORED (for compatibility with built-in rust test runner)" + )] + ignored: bool, } impl Options { @@ -354,15 +375,15 @@ impl Options { /// Logs warning messages to stdout if any ignored options are passed fn warn_on_ignored(&self) { if self.format.is_some() { - println!("WARNING: Ignoring `--format` compatibility option"); + eprintln!("WARNING: Ignoring `--format` compatibility option"); } if self.z_options.is_some() { - println!("WARNING: Ignoring `-Z` compatibility option"); + eprintln!("WARNING: Ignoring `-Z` compatibility option"); } if self.show_output { - println!("WARNING: Ignoring `--show-output` compatibility option"); + eprintln!("WARNING: Ignoring `--show-output` compatibility option"); } } } From 73cfa6c266763b3db15942e3f331f3d5274169c1 Mon Sep 17 00:00:00 2001 From: Leslie Su <3530611790@qq.com> Date: Sat, 26 Oct 2024 18:29:37 +0800 Subject: [PATCH 21/27] feat: Add `Date32`/`Date64` in aggregate fuzz testing (#13041) * refactor PrimitiveArrayGenerator. * support Date32/Date64 type in data generator. * fix format. * remove unnecessary type para in PrimitiveArrayGenerator. * introduce FromNative trait and replace the unsafe. --- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 2 + .../aggregation_fuzzer/data_generator.rs | 46 ++++-- test-utils/src/array_gen/primitive.rs | 134 ++++++++++++------ 3 files changed, 126 insertions(+), 56 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 1035fa31da08..28901b14b5b7 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -164,6 +164,8 @@ fn baseline_config() -> DatasetGeneratorConfig { ColumnDescr::new("u16", DataType::UInt16), ColumnDescr::new("u32", DataType::UInt32), ColumnDescr::new("u64", DataType::UInt64), + ColumnDescr::new("date32", DataType::Date32), + ColumnDescr::new("date64", DataType::Date64), // TODO: date/time columns // todo decimal columns // begin string columns diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs index 44f96d5a1a07..ef9b5a7f355a 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs @@ -17,6 +17,10 @@ use std::sync::Arc; +use arrow::datatypes::{ + Date32Type, Date64Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, + Int8Type, UInt16Type, UInt32Type, UInt64Type, UInt8Type, +}; use arrow_array::{ArrayRef, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{arrow_datafusion_err, DataFusionError, Result}; @@ -222,7 +226,7 @@ macro_rules! generate_string_array { } macro_rules! generate_primitive_array { - ($SELF:ident, $NUM_ROWS:ident, $BATCH_GEN_RNG:ident, $ARRAY_GEN_RNG:ident, $DATA_TYPE:ident) => { + ($SELF:ident, $NUM_ROWS:ident, $BATCH_GEN_RNG:ident, $ARRAY_GEN_RNG:ident, $ARROW_TYPE:ident) => { paste::paste! {{ let null_pct_idx = $BATCH_GEN_RNG.gen_range(0..$SELF.candidate_null_pcts.len()); let null_pct = $SELF.candidate_null_pcts[null_pct_idx]; @@ -239,7 +243,7 @@ macro_rules! generate_primitive_array { rng: $ARRAY_GEN_RNG, }; - generator.[< gen_data_ $DATA_TYPE >]() + generator.gen_data::<$ARROW_TYPE>() }}} } @@ -297,7 +301,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - i8 + Int8Type ) } DataType::Int16 => { @@ -306,7 +310,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - i16 + Int16Type ) } DataType::Int32 => { @@ -315,7 +319,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - i32 + Int32Type ) } DataType::Int64 => { @@ -324,7 +328,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - i64 + Int64Type ) } DataType::UInt8 => { @@ -333,7 +337,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - u8 + UInt8Type ) } DataType::UInt16 => { @@ -342,7 +346,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - u16 + UInt16Type ) } DataType::UInt32 => { @@ -351,7 +355,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - u32 + UInt32Type ) } DataType::UInt64 => { @@ -360,7 +364,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - u64 + UInt64Type ) } DataType::Float32 => { @@ -369,7 +373,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - f32 + Float32Type ) } DataType::Float64 => { @@ -378,7 +382,25 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - f64 + Float64Type + ) + } + DataType::Date32 => { + generate_primitive_array!( + self, + num_rows, + batch_gen_rng, + array_gen_rng, + Date32Type + ) + } + DataType::Date64 => { + generate_primitive_array!( + self, + num_rows, + batch_gen_rng, + array_gen_rng, + Date64Type ) } DataType::Utf8 => { diff --git a/test-utils/src/array_gen/primitive.rs b/test-utils/src/array_gen/primitive.rs index f70ebf6686d0..0581862d63bd 100644 --- a/test-utils/src/array_gen/primitive.rs +++ b/test-utils/src/array_gen/primitive.rs @@ -15,14 +15,45 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::{ArrayRef, PrimitiveArray, UInt32Array}; -use arrow::datatypes::{ - Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, - UInt32Type, UInt64Type, UInt8Type, -}; +use arrow::array::{ArrayRef, ArrowPrimitiveType, PrimitiveArray, UInt32Array}; +use arrow::datatypes::DataType; +use rand::distributions::Standard; +use rand::prelude::Distribution; use rand::rngs::StdRng; use rand::Rng; +/// Trait for converting type safely from a native type T impl this trait. +pub trait FromNative: std::fmt::Debug + Send + Sync + Copy + Default { + /// Convert native type from i64. + fn from_i64(_: i64) -> Option { + None + } +} + +macro_rules! native_type { + ($t: ty $(, $from:ident)*) => { + impl FromNative for $t { + $( + #[inline] + fn $from(v: $t) -> Option { + Some(v) + } + )* + } + }; +} + +native_type!(i8); +native_type!(i16); +native_type!(i32); +native_type!(i64, from_i64); +native_type!(u8); +native_type!(u16); +native_type!(u32); +native_type!(u64); +native_type!(f32); +native_type!(f64); + /// Randomly generate primitive array pub struct PrimitiveArrayGenerator { /// the total number of strings in the output @@ -35,46 +66,61 @@ pub struct PrimitiveArrayGenerator { pub rng: StdRng, } -macro_rules! impl_gen_data { - ($NATIVE_TYPE:ty, $ARROW_TYPE:ident) => { - paste::paste! { - pub fn [< gen_data_ $NATIVE_TYPE >](&mut self) -> ArrayRef { - // table of strings from which to draw - let distinct_primitives: PrimitiveArray<$ARROW_TYPE> = (0..self.num_distinct_primitives) - .map(|_| Some(self.rng.gen::<$NATIVE_TYPE>())) - .collect(); +// TODO: support generating more primitive arrays +impl PrimitiveArrayGenerator { + pub fn gen_data(&mut self) -> ArrayRef + where + A: ArrowPrimitiveType, + A::Native: FromNative, + Standard: Distribution<::Native>, + { + // table of primitives from which to draw + let distinct_primitives: PrimitiveArray = (0..self.num_distinct_primitives) + .map(|_| { + Some(match A::DATA_TYPE { + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::UInt8 + | DataType::UInt16 + | DataType::UInt32 + | DataType::UInt64 + | DataType::Float32 + | DataType::Float64 + | DataType::Date32 => self.rng.gen::(), - // pick num_strings randomly from the distinct string table - let indicies: UInt32Array = (0..self.num_primitives) - .map(|_| { - if self.rng.gen::() < self.null_pct { - None - } else if self.num_distinct_primitives > 1 { - let range = 1..(self.num_distinct_primitives as u32); - Some(self.rng.gen_range(range)) - } else { - Some(0) - } - }) - .collect(); + DataType::Date64 => { + // TODO: constrain this range to valid dates if necessary + let date_value = self.rng.gen_range(i64::MIN..=i64::MAX); + let millis_per_day = 86_400_000; + let adjusted_value = date_value - (date_value % millis_per_day); + A::Native::from_i64(adjusted_value).unwrap() + } - let options = None; - arrow::compute::take(&distinct_primitives, &indicies, options).unwrap() - } - } - }; -} + _ => { + let arrow_type = A::DATA_TYPE; + panic!("Unsupported arrow data type: {arrow_type}") + } + }) + }) + .collect(); -// TODO: support generating more primitive arrays -impl PrimitiveArrayGenerator { - impl_gen_data!(i8, Int8Type); - impl_gen_data!(i16, Int16Type); - impl_gen_data!(i32, Int32Type); - impl_gen_data!(i64, Int64Type); - impl_gen_data!(u8, UInt8Type); - impl_gen_data!(u16, UInt16Type); - impl_gen_data!(u32, UInt32Type); - impl_gen_data!(u64, UInt64Type); - impl_gen_data!(f32, Float32Type); - impl_gen_data!(f64, Float64Type); + // pick num_primitves randomly from the distinct string table + let indicies: UInt32Array = (0..self.num_primitives) + .map(|_| { + if self.rng.gen::() < self.null_pct { + None + } else if self.num_distinct_primitives > 1 { + let range = 1..(self.num_distinct_primitives as u32); + Some(self.rng.gen_range(range)) + } else { + Some(0) + } + }) + .collect(); + + let options = None; + arrow::compute::take(&distinct_primitives, &indicies, options).unwrap() + } } From 22a242c15deafc78f5e6b42eb98408181979cf00 Mon Sep 17 00:00:00 2001 From: Jonathan Chen Date: Sat, 26 Oct 2024 06:30:34 -0400 Subject: [PATCH 22/27] docs: Added Special Functions Page (#13102) * Added special function page * Add index entry, tweak wording * Improve example * Update docs/source/user-guide/sql/special_functions.md --------- Co-authored-by: Andrew Lamb Co-authored-by: Oleks V --- docs/source/user-guide/sql/index.rst | 1 + .../source/user-guide/sql/scalar_functions.md | 77 -------------- .../user-guide/sql/special_functions.md | 100 ++++++++++++++++++ 3 files changed, 101 insertions(+), 77 deletions(-) create mode 100644 docs/source/user-guide/sql/special_functions.md diff --git a/docs/source/user-guide/sql/index.rst b/docs/source/user-guide/sql/index.rst index 6eb451c83b96..8b8afc7b048a 100644 --- a/docs/source/user-guide/sql/index.rst +++ b/docs/source/user-guide/sql/index.rst @@ -35,5 +35,6 @@ SQL Reference window_functions_new scalar_functions scalar_functions_new + special_functions sql_status write_options diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 203411428777..a8e25930bef7 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -72,51 +72,8 @@ See [date_part](#date_part). ## Array Functions -- [unnest](#unnest) - [range](#range) -### `unnest` - -Transforms an array into rows. - -#### Arguments - -- **array**: Array expression to unnest. - Can be a constant, column, or function, and any combination of array operators. - -#### Examples - -``` -> select unnest(make_array(1, 2, 3, 4, 5)); -+------------------------------------------------------------------+ -| unnest(make_array(Int64(1),Int64(2),Int64(3),Int64(4),Int64(5))) | -+------------------------------------------------------------------+ -| 1 | -| 2 | -| 3 | -| 4 | -| 5 | -+------------------------------------------------------------------+ -``` - -``` -> select unnest(range(0, 10)); -+-----------------------------------+ -| unnest(range(Int64(0),Int64(10))) | -+-----------------------------------+ -| 0 | -| 1 | -| 2 | -| 3 | -| 4 | -| 5 | -| 6 | -| 7 | -| 8 | -| 9 | -+-----------------------------------+ -``` - ### `range` Returns an Arrow array between start and stop with step. `SELECT range(2, 10, 3) -> [2, 5, 8]` or @@ -165,40 +122,6 @@ are not allowed - generate_series -## Struct Functions - -- [unnest](#unnest-struct) - -For more struct functions see the new documentation [ -`here`](https://datafusion.apache.org/user-guide/sql/scalar_functions_new.html) - -### `unnest (struct)` - -Unwraps struct fields into columns. - -#### Arguments - -- **struct**: Object expression to unnest. - Can be a constant, column, or function, and any combination of object operators. - -#### Examples - -``` -> select * from foo; -+---------------------+ -| column1 | -+---------------------+ -| {a: 5, b: a string} | -+---------------------+ - -> select unnest(column1) from foo; -+-----------------------+-----------------------+ -| unnest(foo.column1).a | unnest(foo.column1).b | -+-----------------------+-----------------------+ -| 5 | a string | -+-----------------------+-----------------------+ -``` - ## Other Functions See the new documentation [`here`](https://datafusion.apache.org/user-guide/sql/scalar_functions_new.html) diff --git a/docs/source/user-guide/sql/special_functions.md b/docs/source/user-guide/sql/special_functions.md new file mode 100644 index 000000000000..7c9efbb66218 --- /dev/null +++ b/docs/source/user-guide/sql/special_functions.md @@ -0,0 +1,100 @@ + + +# Special Functions + +## Expansion Functions + +- [unnest](#unnest) +- [unnest(struct)](#unnest-struct) + +### `unnest` + +Expands an array or map into rows. + +#### Arguments + +- **array**: Array expression to unnest. + Can be a constant, column, or function, and any combination of array operators. + +#### Examples + +```sql +> select unnest(make_array(1, 2, 3, 4, 5)) as unnested; ++----------+ +| unnested | ++----------+ +| 1 | +| 2 | +| 3 | +| 4 | +| 5 | ++----------+ +``` + +```sql +> select unnest(range(0, 10)) as unnested_range; ++----------------+ +| unnested_range | ++----------------+ +| 0 | +| 1 | +| 2 | +| 3 | +| 4 | +| 5 | +| 6 | +| 7 | +| 8 | +| 9 | ++----------------+ +``` + +### `unnest (struct)` + +Expand a struct fields into individual columns. + +#### Arguments + +- **struct**: Object expression to unnest. + Can be a constant, column, or function, and any combination of object operators. + +#### Examples + +```sql +> create table foo as values ({a: 5, b: 'a string'}), ({a:6, b: 'another string'}); + +> create view foov as select column1 as struct_column from foo; + +> select * from foov; ++---------------------------+ +| struct_column | ++---------------------------+ +| {a: 5, b: a string} | +| {a: 6, b: another string} | ++---------------------------+ + +> select unnest(struct_column) from foov; ++------------------------------------------+------------------------------------------+ +| unnest_placeholder(foov.struct_column).a | unnest_placeholder(foov.struct_column).b | ++------------------------------------------+------------------------------------------+ +| 5 | a string | +| 6 | another string | ++------------------------------------------+------------------------------------------+ +``` From d2511b258e1d3e286ca50a21e671633e2281b105 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Sat, 26 Oct 2024 18:41:26 +0800 Subject: [PATCH 23/27] fix: planning of prepare statement with limit clause (#13088) * fix: planning of prepare statement with limit clause * Improve test --- datafusion/sql/src/query.rs | 10 ++++++---- datafusion/sql/tests/sql_integration.rs | 23 +++++++++++++++++++++++ 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index 842a1c0cbec1..1ef009132f9e 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -53,7 +53,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // so we need to process `SELECT` and `ORDER BY` together. let oby_exprs = to_order_by_exprs(query.order_by)?; let plan = self.select_to_plan(*select, oby_exprs, planner_context)?; - let plan = self.limit(plan, query.offset, query.limit)?; + let plan = + self.limit(plan, query.offset, query.limit, planner_context)?; // Process the `SELECT INTO` after `LIMIT`. self.select_into(plan, select_into) } @@ -68,7 +69,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { None, )?; let plan = self.order_by(plan, order_by_rex)?; - self.limit(plan, query.offset, query.limit) + self.limit(plan, query.offset, query.limit, planner_context) } } } @@ -79,6 +80,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { input: LogicalPlan, skip: Option, fetch: Option, + planner_context: &mut PlannerContext, ) -> Result { if skip.is_none() && fetch.is_none() { return Ok(input); @@ -88,10 +90,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let empty_schema = DFSchema::empty(); let skip = skip - .map(|o| self.sql_to_expr(o.value, &empty_schema, &mut PlannerContext::new())) + .map(|o| self.sql_to_expr(o.value, &empty_schema, planner_context)) .transpose()?; let fetch = fetch - .map(|e| self.sql_to_expr(e, &empty_schema, &mut PlannerContext::new())) + .map(|e| self.sql_to_expr(e, &empty_schema, planner_context)) .transpose()?; LogicalPlanBuilder::from(input) .limit_by_expr(skip, fetch)? diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index edb614493b38..698c408e538f 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -4209,6 +4209,29 @@ fn test_prepare_statement_to_plan_having() { prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); } +#[test] +fn test_prepare_statement_to_plan_limit() { + let sql = "PREPARE my_plan(BIGINT, BIGINT) AS + SELECT id FROM person \ + OFFSET $1 LIMIT $2"; + + let expected_plan = "Prepare: \"my_plan\" [Int64, Int64] \ + \n Limit: skip=$1, fetch=$2\ + \n Projection: person.id\ + \n TableScan: person"; + + let expected_dt = "[Int64, Int64]"; + + let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); + + // replace params with values + let param_values = vec![ScalarValue::Int64(Some(10)), ScalarValue::Int64(Some(200))]; + let expected_plan = "Limit: skip=10, fetch=200\ + \n Projection: person.id\ + \n TableScan: person"; + prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); +} + #[test] fn test_prepare_statement_to_plan_value_list() { let sql = "PREPARE my_plan(STRING, STRING) AS SELECT * FROM (VALUES(1, $1), (2, $2)) AS t (num, letter);"; From 7df3e5cd11f63226b90783564ae7268ee2512ec1 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Sat, 26 Oct 2024 18:59:09 +0800 Subject: [PATCH 24/27] Add benchmark for memory-limited aggregation (#13090) * Adding benchmark for external aggregation * comments --- benchmarks/README.md | 28 ++ benchmarks/bench.sh | 25 +- benchmarks/src/bin/external_aggr.rs | 390 ++++++++++++++++++++ datafusion/execution/src/memory_pool/mod.rs | 12 +- 4 files changed, 450 insertions(+), 5 deletions(-) create mode 100644 benchmarks/src/bin/external_aggr.rs diff --git a/benchmarks/README.md b/benchmarks/README.md index a12662ccb846..a9aa1afb97a1 100644 --- a/benchmarks/README.md +++ b/benchmarks/README.md @@ -352,6 +352,34 @@ This benchmarks is derived from the [TPC-H][1] version [2]: https://github.com/databricks/tpch-dbgen.git, [2.17.1]: https://www.tpc.org/tpc_documents_current_versions/pdf/tpc-h_v2.17.1.pdf +## External Aggregation + +Run the benchmark for aggregations with limited memory. + +When the memory limit is exceeded, the aggregation intermediate results will be spilled to disk, and finally read back with sort-merge. + +External aggregation benchmarks run several aggregation queries with different memory limits, on TPCH `lineitem` table. Queries can be found in [`external_aggr.rs`](src/bin/external_aggr.rs). + +This benchmark is inspired by [DuckDB's external aggregation paper](https://hannes.muehleisen.org/publications/icde2024-out-of-core-kuiper-boncz-muehleisen.pdf), specifically Section VI. + +### External Aggregation Example Runs +1. Run all queries with predefined memory limits: +```bash +# Under 'benchmarks/' directory +cargo run --release --bin external_aggr -- benchmark -n 4 --iterations 3 -p '....../data/tpch_sf1' -o '/tmp/aggr.json' +``` + +2. Run a query with specific memory limit: +```bash +cargo run --release --bin external_aggr -- benchmark -n 4 --iterations 3 -p '....../data/tpch_sf1' -o '/tmp/aggr.json' --query 1 --memory-limit 30M +``` + +3. Run all queries with `bench.sh` script: +```bash +./bench.sh data external_aggr +./bench.sh run external_aggr +``` + # Older Benchmarks diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index fc10cc5afc53..47c5d1261605 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -78,6 +78,7 @@ sort: Benchmark of sorting speed clickbench_1: ClickBench queries against a single parquet file clickbench_partitioned: ClickBench queries against a partitioned (100 files) parquet clickbench_extended: ClickBench \"inspired\" queries against a single parquet (DataFusion specific) +external_aggr: External aggregation benchmark ********** * Supported Configuration (Environment Variables) @@ -170,6 +171,10 @@ main() { imdb) data_imdb ;; + external_aggr) + # same data as for tpch + data_tpch "1" + ;; *) echo "Error: unknown benchmark '$BENCHMARK' for data generation" usage @@ -212,6 +217,7 @@ main() { run_clickbench_partitioned run_clickbench_extended run_imdb + run_external_aggr ;; tpch) run_tpch "1" @@ -243,6 +249,9 @@ main() { imdb) run_imdb ;; + external_aggr) + run_external_aggr + ;; *) echo "Error: unknown benchmark '$BENCHMARK' for run" usage @@ -524,7 +533,21 @@ run_imdb() { $CARGO_COMMAND --bin imdb -- benchmark datafusion --iterations 5 --path "${IMDB_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --format parquet -o "${RESULTS_FILE}" } - +# Runs the external aggregation benchmark +run_external_aggr() { + # Use TPC-H SF1 dataset + TPCH_DIR="${DATA_DIR}/tpch_sf1" + RESULTS_FILE="${RESULTS_DIR}/external_aggr.json" + echo "RESULTS_FILE: ${RESULTS_FILE}" + echo "Running external aggregation benchmark..." + + # Only parquet is supported. + # Since per-operator memory limit is calculated as (total-memory-limit / + # number-of-partitions), and by default `--partitions` is set to number of + # CPU cores, we set a constant number of partitions to prevent this + # benchmark to fail on some machines. + $CARGO_COMMAND --bin external_aggr -- benchmark --partitions 4 --iterations 5 --path "${TPCH_DIR}" -o "${RESULTS_FILE}" +} compare_benchmarks() { diff --git a/benchmarks/src/bin/external_aggr.rs b/benchmarks/src/bin/external_aggr.rs new file mode 100644 index 000000000000..1bc74e22ccfa --- /dev/null +++ b/benchmarks/src/bin/external_aggr.rs @@ -0,0 +1,390 @@ +// 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. + +//! external_aggr binary entrypoint + +use std::collections::HashMap; +use std::path::PathBuf; +use std::sync::Arc; +use std::sync::OnceLock; +use structopt::StructOpt; + +use arrow::record_batch::RecordBatch; +use arrow::util::pretty; +use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::listing::{ + ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, +}; +use datafusion::datasource::{MemTable, TableProvider}; +use datafusion::error::Result; +use datafusion::execution::memory_pool::FairSpillPool; +use datafusion::execution::memory_pool::{human_readable_size, units}; +use datafusion::execution::runtime_env::RuntimeConfig; +use datafusion::physical_plan::display::DisplayableExecutionPlan; +use datafusion::physical_plan::{collect, displayable}; +use datafusion::prelude::*; +use datafusion_benchmarks::util::{BenchmarkRun, CommonOpt}; +use datafusion_common::instant::Instant; +use datafusion_common::{exec_datafusion_err, exec_err, DEFAULT_PARQUET_EXTENSION}; + +#[derive(Debug, StructOpt)] +#[structopt( + name = "datafusion-external-aggregation", + about = "DataFusion external aggregation benchmark" +)] +enum ExternalAggrOpt { + Benchmark(ExternalAggrConfig), +} + +#[derive(Debug, StructOpt)] +struct ExternalAggrConfig { + /// Query number. If not specified, runs all queries + #[structopt(short, long)] + query: Option, + + /// Memory limit (e.g. '100M', '1.5G'). If not specified, run all pre-defined memory limits for given query. + #[structopt(long)] + memory_limit: Option, + + /// Common options + #[structopt(flatten)] + common: CommonOpt, + + /// Path to data files (lineitem). Only parquet format is supported + #[structopt(parse(from_os_str), required = true, short = "p", long = "path")] + path: PathBuf, + + /// Load the data into a MemTable before executing the query + #[structopt(short = "m", long = "mem-table")] + mem_table: bool, + + /// Path to JSON benchmark result to be compare using `compare.py` + #[structopt(parse(from_os_str), short = "o", long = "output")] + output_path: Option, +} + +struct QueryResult { + elapsed: std::time::Duration, + row_count: usize, +} + +/// Query Memory Limits +/// Map query id to predefined memory limits +/// +/// Q1 requires 36MiB for aggregation +/// Memory limits to run: 64MiB, 32MiB, 16MiB +/// Q2 requires 250MiB for aggregation +/// Memory limits to run: 512MiB, 256MiB, 128MiB, 64MiB, 32MiB +static QUERY_MEMORY_LIMITS: OnceLock>> = OnceLock::new(); + +impl ExternalAggrConfig { + const AGGR_TABLES: [&'static str; 1] = ["lineitem"]; + const AGGR_QUERIES: [&'static str; 2] = [ + // Q1: Output size is ~25% of lineitem table + r#" + SELECT count(*) + FROM ( + SELECT DISTINCT l_orderkey + FROM lineitem + ) + "#, + // Q2: Output size is ~99% of lineitem table + r#" + SELECT count(*) + FROM ( + SELECT DISTINCT l_orderkey, l_suppkey + FROM lineitem + ) + "#, + ]; + + fn init_query_memory_limits() -> &'static HashMap> { + use units::*; + QUERY_MEMORY_LIMITS.get_or_init(|| { + let mut map = HashMap::new(); + map.insert(1, vec![64 * MB, 32 * MB, 16 * MB]); + map.insert(2, vec![512 * MB, 256 * MB, 128 * MB, 64 * MB, 32 * MB]); + map + }) + } + + /// If `--query` and `--memory-limit` is not speicified, run all queries + /// with pre-configured memory limits + /// If only `--query` is specified, run the query with all memory limits + /// for this query + /// If both `--query` and `--memory-limit` are specified, run the query + /// with the specified memory limit + pub async fn run(&self) -> Result<()> { + let mut benchmark_run = BenchmarkRun::new(); + + let memory_limit = match &self.memory_limit { + Some(limit) => Some(Self::parse_memory_limit(limit)?), + None => None, + }; + + let query_range = match self.query { + Some(query_id) => query_id..=query_id, + None => 1..=Self::AGGR_QUERIES.len(), + }; + + // Each element is (query_id, memory_limit) + // e.g. [(1, 64_000), (1, 32_000)...] means first run Q1 with 64KiB + // memory limit, next run Q1 with 32KiB memory limit, etc. + let mut query_executions = vec![]; + // Setup `query_executions` + for query_id in query_range { + if query_id > Self::AGGR_QUERIES.len() { + return exec_err!( + "Invalid '--query'(query number) {} for external aggregation benchmark.", + query_id + ); + } + + match memory_limit { + Some(limit) => { + query_executions.push((query_id, limit)); + } + None => { + let memory_limits_table = Self::init_query_memory_limits(); + let memory_limits = memory_limits_table.get(&query_id).unwrap(); + for limit in memory_limits { + query_executions.push((query_id, *limit)); + } + } + } + } + + for (query_id, mem_limit) in query_executions { + benchmark_run.start_new_case(&format!( + "{query_id}({})", + human_readable_size(mem_limit as usize) + )); + + let query_results = self.benchmark_query(query_id, mem_limit).await?; + for iter in query_results { + benchmark_run.write_iter(iter.elapsed, iter.row_count); + } + } + + benchmark_run.maybe_write_json(self.output_path.as_ref())?; + + Ok(()) + } + + /// Benchmark query `query_id` in `AGGR_QUERIES` + async fn benchmark_query( + &self, + query_id: usize, + mem_limit: u64, + ) -> Result> { + let query_name = + format!("Q{query_id}({})", human_readable_size(mem_limit as usize)); + let mut config = self.common.config(); + config + .options_mut() + .execution + .parquet + .schema_force_view_types = self.common.force_view_types; + let runtime_config = RuntimeConfig::new() + .with_memory_pool(Arc::new(FairSpillPool::new(mem_limit as usize))) + .build_arc()?; + let ctx = SessionContext::new_with_config_rt(config, runtime_config); + + // register tables + self.register_tables(&ctx).await?; + + let mut millis = vec![]; + // run benchmark + let mut query_results = vec![]; + for i in 0..self.iterations() { + let start = Instant::now(); + + let query_idx = query_id - 1; // 1-indexed -> 0-indexed + let sql = Self::AGGR_QUERIES[query_idx]; + + let result = self.execute_query(&ctx, sql).await?; + + let elapsed = start.elapsed(); //.as_secs_f64() * 1000.0; + let ms = elapsed.as_secs_f64() * 1000.0; + millis.push(ms); + + let row_count = result.iter().map(|b| b.num_rows()).sum(); + println!( + "{query_name} iteration {i} took {ms:.1} ms and returned {row_count} rows" + ); + query_results.push(QueryResult { elapsed, row_count }); + } + + let avg = millis.iter().sum::() / millis.len() as f64; + println!("{query_name} avg time: {avg:.2} ms"); + + Ok(query_results) + } + + async fn register_tables(&self, ctx: &SessionContext) -> Result<()> { + for table in Self::AGGR_TABLES { + let table_provider = { self.get_table(ctx, table).await? }; + + if self.mem_table { + println!("Loading table '{table}' into memory"); + let start = Instant::now(); + let memtable = + MemTable::load(table_provider, Some(self.partitions()), &ctx.state()) + .await?; + println!( + "Loaded table '{}' into memory in {} ms", + table, + start.elapsed().as_millis() + ); + ctx.register_table(table, Arc::new(memtable))?; + } else { + ctx.register_table(table, table_provider)?; + } + } + Ok(()) + } + + async fn execute_query( + &self, + ctx: &SessionContext, + sql: &str, + ) -> Result> { + let debug = self.common.debug; + let plan = ctx.sql(sql).await?; + let (state, plan) = plan.into_parts(); + + if debug { + println!("=== Logical plan ===\n{plan}\n"); + } + + let plan = state.optimize(&plan)?; + if debug { + println!("=== Optimized logical plan ===\n{plan}\n"); + } + let physical_plan = state.create_physical_plan(&plan).await?; + if debug { + println!( + "=== Physical plan ===\n{}\n", + displayable(physical_plan.as_ref()).indent(true) + ); + } + let result = collect(physical_plan.clone(), state.task_ctx()).await?; + if debug { + println!( + "=== Physical plan with metrics ===\n{}\n", + DisplayableExecutionPlan::with_metrics(physical_plan.as_ref()) + .indent(true) + ); + if !result.is_empty() { + // do not call print_batches if there are no batches as the result is confusing + // and makes it look like there is a batch with no columns + pretty::print_batches(&result)?; + } + } + Ok(result) + } + + async fn get_table( + &self, + ctx: &SessionContext, + table: &str, + ) -> Result> { + let path = self.path.to_str().unwrap(); + + // Obtain a snapshot of the SessionState + let state = ctx.state(); + let path = format!("{path}/{table}"); + let format = Arc::new( + ParquetFormat::default() + .with_options(ctx.state().table_options().parquet.clone()), + ); + let extension = DEFAULT_PARQUET_EXTENSION; + + let options = ListingOptions::new(format) + .with_file_extension(extension) + .with_collect_stat(state.config().collect_statistics()); + + let table_path = ListingTableUrl::parse(path)?; + let config = ListingTableConfig::new(table_path).with_listing_options(options); + let config = config.infer_schema(&state).await?; + + Ok(Arc::new(ListingTable::try_new(config)?)) + } + + fn iterations(&self) -> usize { + self.common.iterations + } + + fn partitions(&self) -> usize { + self.common.partitions.unwrap_or(num_cpus::get()) + } + + /// Parse memory limit from string to number of bytes + /// e.g. '1.5G', '100M' -> 1572864 + fn parse_memory_limit(limit: &str) -> Result { + let (number, unit) = limit.split_at(limit.len() - 1); + let number: f64 = number.parse().map_err(|_| { + exec_datafusion_err!("Failed to parse number from memory limit '{}'", limit) + })?; + + match unit { + "K" => Ok((number * 1024.0) as u64), + "M" => Ok((number * 1024.0 * 1024.0) as u64), + "G" => Ok((number * 1024.0 * 1024.0 * 1024.0) as u64), + _ => exec_err!("Unsupported unit '{}' in memory limit '{}'", unit, limit), + } + } +} + +#[tokio::main] +pub async fn main() -> Result<()> { + env_logger::init(); + + match ExternalAggrOpt::from_args() { + ExternalAggrOpt::Benchmark(opt) => opt.run().await?, + } + + Ok(()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_parse_memory_limit_all() { + // Test valid inputs + assert_eq!( + ExternalAggrConfig::parse_memory_limit("100K").unwrap(), + 102400 + ); + assert_eq!( + ExternalAggrConfig::parse_memory_limit("1.5M").unwrap(), + 1572864 + ); + assert_eq!( + ExternalAggrConfig::parse_memory_limit("2G").unwrap(), + 2147483648 + ); + + // Test invalid unit + assert!(ExternalAggrConfig::parse_memory_limit("500X").is_err()); + + // Test invalid number + assert!(ExternalAggrConfig::parse_memory_limit("abcM").is_err()); + } +} diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index d87ce1ebfed7..5bf30b724d0b 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -334,13 +334,17 @@ impl Drop for MemoryReservation { } } -const TB: u64 = 1 << 40; -const GB: u64 = 1 << 30; -const MB: u64 = 1 << 20; -const KB: u64 = 1 << 10; +pub mod units { + pub const TB: u64 = 1 << 40; + pub const GB: u64 = 1 << 30; + pub const MB: u64 = 1 << 20; + pub const KB: u64 = 1 << 10; +} /// Present size in human readable form pub fn human_readable_size(size: usize) -> String { + use units::*; + let size = size as u64; let (value, unit) = { if size >= 2 * TB { From 412ca4e1ffe0a005cb772393f1e920ab395a6200 Mon Sep 17 00:00:00 2001 From: Bruce Ritchie Date: Sat, 26 Oct 2024 07:38:32 -0400 Subject: [PATCH 25/27] Add clickbench parquet based queries to sql_planner benchmark (#13103) * Add clickbench parquet based queries to sql_planner benchmark. * Cargo fmt. * Commented out most logical_plan tests & updated code to allow for running from either cargo or via target/release/deps/sql_planner-xyz --- datafusion/core/benches/sql_planner.rs | 121 ++++++++++++++++++++++--- 1 file changed, 107 insertions(+), 14 deletions(-) diff --git a/datafusion/core/benches/sql_planner.rs b/datafusion/core/benches/sql_planner.rs index 6f9cf02873d1..140e266a0272 100644 --- a/datafusion/core/benches/sql_planner.rs +++ b/datafusion/core/benches/sql_planner.rs @@ -15,22 +15,31 @@ // specific language governing permissions and limitations // under the License. +extern crate arrow; #[macro_use] extern crate criterion; -extern crate arrow; extern crate datafusion; mod data_utils; + use crate::criterion::Criterion; use arrow::datatypes::{DataType, Field, Fields, Schema}; use datafusion::datasource::MemTable; use datafusion::execution::context::SessionContext; +use itertools::Itertools; +use std::fs::File; +use std::io::{BufRead, BufReader}; +use std::path::PathBuf; use std::sync::Arc; use test_utils::tpcds::tpcds_schemas; use test_utils::tpch::tpch_schemas; use test_utils::TableDef; use tokio::runtime::Runtime; +const BENCHMARKS_PATH_1: &str = "../../benchmarks/"; +const BENCHMARKS_PATH_2: &str = "./benchmarks/"; +const CLICKBENCH_DATA_PATH: &str = "data/hits_partitioned/"; + /// Create a logical plan from the specified sql fn logical_plan(ctx: &SessionContext, sql: &str) { let rt = Runtime::new().unwrap(); @@ -91,7 +100,37 @@ fn register_defs(ctx: SessionContext, defs: Vec) -> SessionContext { ctx } +fn register_clickbench_hits_table() -> SessionContext { + let ctx = SessionContext::new(); + let rt = Runtime::new().unwrap(); + + // use an external table for clickbench benchmarks + let path = + if PathBuf::from(format!("{BENCHMARKS_PATH_1}{CLICKBENCH_DATA_PATH}")).exists() { + format!("{BENCHMARKS_PATH_1}{CLICKBENCH_DATA_PATH}") + } else { + format!("{BENCHMARKS_PATH_2}{CLICKBENCH_DATA_PATH}") + }; + + let sql = format!("CREATE EXTERNAL TABLE hits STORED AS PARQUET LOCATION '{path}'"); + + rt.block_on(ctx.sql(&sql)).unwrap(); + + let count = + rt.block_on(async { ctx.table("hits").await.unwrap().count().await.unwrap() }); + assert!(count > 0); + ctx +} + fn criterion_benchmark(c: &mut Criterion) { + // verify that we can load the clickbench data prior to running the benchmark + if !PathBuf::from(format!("{BENCHMARKS_PATH_1}{CLICKBENCH_DATA_PATH}")).exists() + && !PathBuf::from(format!("{BENCHMARKS_PATH_2}{CLICKBENCH_DATA_PATH}")).exists() + { + panic!("benchmarks/data/hits_partitioned/ could not be loaded. Please run \ + 'benchmarks/bench.sh data clickbench_partitioned' prior to running this benchmark") + } + let ctx = create_context(); // Test simplest @@ -235,9 +274,15 @@ fn criterion_benchmark(c: &mut Criterion) { "q16", "q17", "q18", "q19", "q20", "q21", "q22", ]; + let benchmarks_path = if PathBuf::from(BENCHMARKS_PATH_1).exists() { + BENCHMARKS_PATH_1 + } else { + BENCHMARKS_PATH_2 + }; + for q in tpch_queries { let sql = - std::fs::read_to_string(format!("../../benchmarks/queries/{q}.sql")).unwrap(); + std::fs::read_to_string(format!("{benchmarks_path}queries/{q}.sql")).unwrap(); c.bench_function(&format!("physical_plan_tpch_{}", q), |b| { b.iter(|| physical_plan(&tpch_ctx, &sql)) }); @@ -246,7 +291,7 @@ fn criterion_benchmark(c: &mut Criterion) { let all_tpch_sql_queries = tpch_queries .iter() .map(|q| { - std::fs::read_to_string(format!("../../benchmarks/queries/{q}.sql")).unwrap() + std::fs::read_to_string(format!("{benchmarks_path}queries/{q}.sql")).unwrap() }) .collect::>(); @@ -258,20 +303,25 @@ fn criterion_benchmark(c: &mut Criterion) { }) }); - c.bench_function("logical_plan_tpch_all", |b| { - b.iter(|| { - for sql in &all_tpch_sql_queries { - logical_plan(&tpch_ctx, sql) - } - }) - }); + // c.bench_function("logical_plan_tpch_all", |b| { + // b.iter(|| { + // for sql in &all_tpch_sql_queries { + // logical_plan(&tpch_ctx, sql) + // } + // }) + // }); // --- TPC-DS --- let tpcds_ctx = register_defs(SessionContext::new(), tpcds_schemas()); + let tests_path = if PathBuf::from("./tests/").exists() { + "./tests/" + } else { + "datafusion/core/tests/" + }; let raw_tpcds_sql_queries = (1..100) - .map(|q| std::fs::read_to_string(format!("./tests/tpc-ds/{q}.sql")).unwrap()) + .map(|q| std::fs::read_to_string(format!("{tests_path}tpc-ds/{q}.sql")).unwrap()) .collect::>(); // some queries have multiple statements @@ -288,10 +338,53 @@ fn criterion_benchmark(c: &mut Criterion) { }) }); - c.bench_function("logical_plan_tpcds_all", |b| { + // c.bench_function("logical_plan_tpcds_all", |b| { + // b.iter(|| { + // for sql in &all_tpcds_sql_queries { + // logical_plan(&tpcds_ctx, sql) + // } + // }) + // }); + + // -- clickbench -- + + let queries_file = + File::open(format!("{benchmarks_path}queries/clickbench/queries.sql")).unwrap(); + let extended_file = + File::open(format!("{benchmarks_path}queries/clickbench/extended.sql")).unwrap(); + + let clickbench_queries: Vec = BufReader::new(queries_file) + .lines() + .chain(BufReader::new(extended_file).lines()) + .map(|l| l.expect("Could not parse line")) + .collect_vec(); + + let clickbench_ctx = register_clickbench_hits_table(); + + // for (i, sql) in clickbench_queries.iter().enumerate() { + // c.bench_function(&format!("logical_plan_clickbench_q{}", i + 1), |b| { + // b.iter(|| logical_plan(&clickbench_ctx, sql)) + // }); + // } + + for (i, sql) in clickbench_queries.iter().enumerate() { + c.bench_function(&format!("physical_plan_clickbench_q{}", i + 1), |b| { + b.iter(|| physical_plan(&clickbench_ctx, sql)) + }); + } + + // c.bench_function("logical_plan_clickbench_all", |b| { + // b.iter(|| { + // for sql in &clickbench_queries { + // logical_plan(&clickbench_ctx, sql) + // } + // }) + // }); + + c.bench_function("physical_plan_clickbench_all", |b| { b.iter(|| { - for sql in &all_tpcds_sql_queries { - logical_plan(&tpcds_ctx, sql) + for sql in &clickbench_queries { + physical_plan(&clickbench_ctx, sql) } }) }); From 62b063cd36653b92a9f0cd53a358231be8c3e848 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 26 Oct 2024 07:41:41 -0400 Subject: [PATCH 26/27] Improve documentation and examples for `SchemaAdapterFactory`, make `record_batch` "hygenic" (#13063) * Improve documentation and examples for SchemaAdapterFactory and related classes * fix macro * Add macro hygene test * Fix example, add convenience function, update docs * Add tests and docs showing what happens when adapting a nullable column * review feedback * fix clippy --- datafusion/common/src/test_util.rs | 2 +- .../core/src/datasource/schema_adapter.rs | 286 ++++++++++++++---- datafusion/core/tests/macro_hygiene/mod.rs | 10 + 3 files changed, 239 insertions(+), 59 deletions(-) diff --git a/datafusion/common/src/test_util.rs b/datafusion/common/src/test_util.rs index 422fcb5eb3e0..d3b8c8451258 100644 --- a/datafusion/common/src/test_util.rs +++ b/datafusion/common/src/test_util.rs @@ -347,7 +347,7 @@ macro_rules! record_batch { let batch = arrow_array::RecordBatch::try_new( schema, vec![$( - create_array!($type, $values), + $crate::create_array!($type, $values), )*] ); diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index 131b8c354ce7..80d2bf987473 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -32,11 +32,19 @@ use std::sync::Arc; /// /// This interface provides a way to implement custom schema adaptation logic /// for ParquetExec (for example, to fill missing columns with default value -/// other than null) +/// other than null). +/// +/// Most users should use [`DefaultSchemaAdapterFactory`]. See that struct for +/// more details and examples. pub trait SchemaAdapterFactory: Debug + Send + Sync + 'static { - /// Provides `SchemaAdapter`. - // The design of this function is mostly modeled for the needs of DefaultSchemaAdapterFactory, - // read its implementation docs for the reasoning + /// Create a [`SchemaAdapter`] + /// + /// Arguments: + /// + /// * `projected_table_schema`: The schema for the table, projected to + /// include only the fields being output (projected) by the this mapping. + /// + /// * `table_schema`: The entire table schema for the table fn create( &self, projected_table_schema: SchemaRef, @@ -44,53 +52,57 @@ pub trait SchemaAdapterFactory: Debug + Send + Sync + 'static { ) -> Box; } -/// Adapt file-level [`RecordBatch`]es to a table schema, which may have a schema -/// obtained from merging multiple file-level schemas. -/// -/// This is useful for enabling schema evolution in partitioned datasets. -/// -/// This has to be done in two stages. +/// Creates [`SchemaMapper`]s to map file-level [`RecordBatch`]es to a table +/// schema, which may have a schema obtained from merging multiple file-level +/// schemas. /// -/// 1. Before reading the file, we have to map projected column indexes from the -/// table schema to the file schema. +/// This is useful for implementing schema evolution in partitioned datasets. /// -/// 2. After reading a record batch map the read columns back to the expected -/// columns indexes and insert null-valued columns wherever the file schema was -/// missing a column present in the table schema. +/// See [`DefaultSchemaAdapterFactory`] for more details and examples. pub trait SchemaAdapter: Send + Sync { /// Map a column index in the table schema to a column index in a particular /// file schema /// + /// This is used while reading a file to push down projections by mapping + /// projected column indexes from the table schema to the file schema + /// /// Panics if index is not in range for the table schema fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option; - /// Creates a `SchemaMapping` that can be used to cast or map the columns - /// from the file schema to the table schema. + /// Creates a mapping for casting columns from the file schema to the table + /// schema. /// - /// If the provided `file_schema` contains columns of a different type to the expected - /// `table_schema`, the method will attempt to cast the array data from the file schema - /// to the table schema where possible. + /// This is used after reading a record batch. The returned [`SchemaMapper`]: /// - /// Returns a [`SchemaMapper`] that can be applied to the output batch - /// along with an ordered list of columns to project from the file + /// 1. Maps columns to the expected columns indexes + /// 2. Handles missing values (e.g. fills nulls or a default value) for + /// columns in the in the table schema not in the file schema + /// 2. Handles different types: if the column in the file schema has a + /// different type than `table_schema`, the mapper will resolve this + /// difference (e.g. by casting to the appropriate type) + /// + /// Returns: + /// * a [`SchemaMapper`] + /// * an ordered list of columns to project from the file fn map_schema( &self, file_schema: &Schema, ) -> datafusion_common::Result<(Arc, Vec)>; } -/// Maps, by casting or reordering columns from the file schema to the table -/// schema. +/// Maps, columns from a specific file schema to the table schema. +/// +/// See [`DefaultSchemaAdapterFactory`] for more details and examples. pub trait SchemaMapper: Debug + Send + Sync { - /// Adapts a `RecordBatch` to match the `table_schema` using the stored - /// mapping and conversions. + /// Adapts a `RecordBatch` to match the `table_schema` fn map_batch(&self, batch: RecordBatch) -> datafusion_common::Result; /// Adapts a [`RecordBatch`] that does not have all the columns from the /// file schema. /// - /// This method is used when applying a filter to a subset of the columns as - /// part of `DataFusionArrowPredicate` when `filter_pushdown` is enabled. + /// This method is used, for example, when applying a filter to a subset of + /// the columns as part of `DataFusionArrowPredicate` when `filter_pushdown` + /// is enabled. /// /// This method is slower than `map_batch` as it looks up columns by name. fn map_partial_batch( @@ -99,11 +111,106 @@ pub trait SchemaMapper: Debug + Send + Sync { ) -> datafusion_common::Result; } -/// Implementation of [`SchemaAdapterFactory`] that maps columns by name -/// and casts columns to the expected type. +/// Default [`SchemaAdapterFactory`] for mapping schemas. +/// +/// This can be used to adapt file-level record batches to a table schema and +/// implement schema evolution. +/// +/// Given an input file schema and a table schema, this factory returns +/// [`SchemaAdapter`] that return [`SchemaMapper`]s that: +/// +/// 1. Reorder columns +/// 2. Cast columns to the correct type +/// 3. Fill missing columns with nulls +/// +/// # Errors: +/// +/// * If a column in the table schema is non-nullable but is not present in the +/// file schema (i.e. it is missing), the returned mapper tries to fill it with +/// nulls resulting in a schema error. +/// +/// # Illustration of Schema Mapping +/// +/// ```text +/// ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// ┌───────┐ ┌───────┐ │ ┌───────┐ ┌───────┐ ┌───────┐ │ +/// ││ 1.0 │ │ "foo" │ ││ NULL │ │ "foo" │ │ "1.0" │ +/// ├───────┤ ├───────┤ │ Schema mapping ├───────┤ ├───────┤ ├───────┤ │ +/// ││ 2.0 │ │ "bar" │ ││ NULL │ │ "bar" │ │ "2.0" │ +/// └───────┘ └───────┘ │────────────────▶ └───────┘ └───────┘ └───────┘ │ +/// │ │ +/// column "c" column "b"│ column "a" column "b" column "c"│ +/// │ Float64 Utf8 │ Int32 Utf8 Utf8 +/// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ +/// Input Record Batch Output Record Batch +/// +/// Schema { Schema { +/// "c": Float64, "a": Int32, +/// "b": Utf8, "b": Utf8, +/// } "c": Utf8, +/// } +/// ``` +/// +/// # Example of using the `DefaultSchemaAdapterFactory` to map [`RecordBatch`]s +/// +/// Note `SchemaMapping` also supports mapping partial batches, which is used as +/// part of predicate pushdown. +/// +/// ``` +/// # use std::sync::Arc; +/// # use arrow::datatypes::{DataType, Field, Schema}; +/// # use datafusion::datasource::schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapterFactory}; +/// # use datafusion_common::record_batch; +/// // Table has fields "a", "b" and "c" +/// let table_schema = Schema::new(vec![ +/// Field::new("a", DataType::Int32, true), +/// Field::new("b", DataType::Utf8, true), +/// Field::new("c", DataType::Utf8, true), +/// ]); +/// +/// // create an adapter to map the table schema to the file schema +/// let adapter = DefaultSchemaAdapterFactory::from_schema(Arc::new(table_schema)); +/// +/// // The file schema has fields "c" and "b" but "b" is stored as an 'Float64' +/// // instead of 'Utf8' +/// let file_schema = Schema::new(vec![ +/// Field::new("c", DataType::Utf8, true), +/// Field::new("b", DataType::Float64, true), +/// ]); +/// +/// // Get a mapping from the file schema to the table schema +/// let (mapper, _indices) = adapter.map_schema(&file_schema).unwrap(); +/// +/// let file_batch = record_batch!( +/// ("c", Utf8, vec!["foo", "bar"]), +/// ("b", Float64, vec![1.0, 2.0]) +/// ).unwrap(); +/// +/// let mapped_batch = mapper.map_batch(file_batch).unwrap(); +/// +/// // the mapped batch has the correct schema and the "b" column has been cast to Utf8 +/// let expected_batch = record_batch!( +/// ("a", Int32, vec![None, None]), // missing column filled with nulls +/// ("b", Utf8, vec!["1.0", "2.0"]), // b was cast to string and order was changed +/// ("c", Utf8, vec!["foo", "bar"]) +/// ).unwrap(); +/// assert_eq!(mapped_batch, expected_batch); +/// ``` #[derive(Clone, Debug, Default)] pub struct DefaultSchemaAdapterFactory; +impl DefaultSchemaAdapterFactory { + /// Create a new factory for mapping batches from a file schema to a table + /// schema. + /// + /// This is a convenience for [`DefaultSchemaAdapterFactory::create`] with + /// the same schema for both the projected table schema and the table + /// schema. + pub fn from_schema(table_schema: SchemaRef) -> Box { + Self.create(Arc::clone(&table_schema), table_schema) + } +} + impl SchemaAdapterFactory for DefaultSchemaAdapterFactory { fn create( &self, @@ -117,8 +224,8 @@ impl SchemaAdapterFactory for DefaultSchemaAdapterFactory { } } -/// This SchemaAdapter requires both the table schema and the projected table schema because of the -/// needs of the [`SchemaMapping`] it creates. Read its documentation for more details +/// This SchemaAdapter requires both the table schema and the projected table +/// schema. See [`SchemaMapping`] for more details #[derive(Clone, Debug)] pub(crate) struct DefaultSchemaAdapter { /// The schema for the table, projected to include only the fields being output (projected) by the @@ -142,11 +249,12 @@ impl SchemaAdapter for DefaultSchemaAdapter { Some(file_schema.fields.find(field.name())?.0) } - /// Creates a `SchemaMapping` that can be used to cast or map the columns from the file schema to the table schema. + /// Creates a `SchemaMapping` for casting or mapping the columns from the + /// file schema to the table schema. /// - /// If the provided `file_schema` contains columns of a different type to the expected - /// `table_schema`, the method will attempt to cast the array data from the file schema - /// to the table schema where possible. + /// If the provided `file_schema` contains columns of a different type to + /// the expected `table_schema`, the method will attempt to cast the array + /// data from the file schema to the table schema where possible. /// /// Returns a [`SchemaMapping`] that can be applied to the output batch /// along with an ordered list of columns to project from the file @@ -189,36 +297,45 @@ impl SchemaAdapter for DefaultSchemaAdapter { } } -/// The SchemaMapping struct holds a mapping from the file schema to the table schema -/// and any necessary type conversions that need to be applied. +/// The SchemaMapping struct holds a mapping from the file schema to the table +/// schema and any necessary type conversions. +/// +/// Note, because `map_batch` and `map_partial_batch` functions have different +/// needs, this struct holds two schemas: +/// +/// 1. The projected **table** schema +/// 2. The full table schema /// -/// This needs both the projected table schema and full table schema because its different -/// functions have different needs. The [`map_batch`] function is only used by the ParquetOpener to -/// produce a RecordBatch which has the projected schema, since that's the schema which is supposed -/// to come out of the execution of this query. [`map_partial_batch`], however, is used to create a -/// RecordBatch with a schema that can be used for Parquet pushdown, meaning that it may contain -/// fields which are not in the projected schema (as the fields that parquet pushdown filters -/// operate can be completely distinct from the fields that are projected (output) out of the -/// ParquetExec). +/// [`map_batch`] is used by the ParquetOpener to produce a RecordBatch which +/// has the projected schema, since that's the schema which is supposed to come +/// out of the execution of this query. Thus `map_batch` uses +/// `projected_table_schema` as it can only operate on the projected fields. /// -/// [`map_partial_batch`] uses `table_schema` to create the resulting RecordBatch (as it could be -/// operating on any fields in the schema), while [`map_batch`] uses `projected_table_schema` (as -/// it can only operate on the projected fields). +/// [`map_partial_batch`] is used to create a RecordBatch with a schema that +/// can be used for Parquet predicate pushdown, meaning that it may contain +/// fields which are not in the projected schema (as the fields that parquet +/// pushdown filters operate can be completely distinct from the fields that are +/// projected (output) out of the ParquetExec). `map_partial_batch` thus uses +/// `table_schema` to create the resulting RecordBatch (as it could be operating +/// on any fields in the schema). /// /// [`map_batch`]: Self::map_batch /// [`map_partial_batch`]: Self::map_partial_batch #[derive(Debug)] pub struct SchemaMapping { - /// The schema of the table. This is the expected schema after conversion and it should match - /// the schema of the query result. + /// The schema of the table. This is the expected schema after conversion + /// and it should match the schema of the query result. projected_table_schema: SchemaRef, - /// Mapping from field index in `projected_table_schema` to index in projected file_schema. - /// They are Options instead of just plain `usize`s because the table could have fields that - /// don't exist in the file. + /// Mapping from field index in `projected_table_schema` to index in + /// projected file_schema. + /// + /// They are Options instead of just plain `usize`s because the table could + /// have fields that don't exist in the file. field_mappings: Vec>, - /// The entire table schema, as opposed to the projected_table_schema (which only contains the - /// columns that we are projecting out of this query). This contains all fields in the table, - /// regardless of if they will be projected out or not. + /// The entire table schema, as opposed to the projected_table_schema (which + /// only contains the columns that we are projecting out of this query). + /// This contains all fields in the table, regardless of if they will be + /// projected out or not. table_schema: SchemaRef, } @@ -331,8 +448,9 @@ mod tests { use crate::datasource::listing::PartitionedFile; use crate::datasource::schema_adapter::{ - SchemaAdapter, SchemaAdapterFactory, SchemaMapper, + DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; + use datafusion_common::record_batch; #[cfg(feature = "parquet")] use parquet::arrow::ArrowWriter; use tempfile::TempDir; @@ -405,6 +523,58 @@ mod tests { assert_batches_sorted_eq!(expected, &read); } + #[test] + fn default_schema_adapter() { + let table_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Utf8, true), + ]); + + // file has a subset of the table schema fields and different type + let file_schema = Schema::new(vec![ + Field::new("c", DataType::Float64, true), // not in table schema + Field::new("b", DataType::Float64, true), + ]); + + let adapter = DefaultSchemaAdapterFactory::from_schema(Arc::new(table_schema)); + let (mapper, indices) = adapter.map_schema(&file_schema).unwrap(); + assert_eq!(indices, vec![1]); + + let file_batch = record_batch!(("b", Float64, vec![1.0, 2.0])).unwrap(); + + let mapped_batch = mapper.map_batch(file_batch).unwrap(); + + // the mapped batch has the correct schema and the "b" column has been cast to Utf8 + let expected_batch = record_batch!( + ("a", Int32, vec![None, None]), // missing column filled with nulls + ("b", Utf8, vec!["1.0", "2.0"]) // b was cast to string and order was changed + ) + .unwrap(); + assert_eq!(mapped_batch, expected_batch); + } + + #[test] + fn default_schema_adapter_non_nullable_columns() { + let table_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), // "a"" is declared non nullable + Field::new("b", DataType::Utf8, true), + ]); + let file_schema = Schema::new(vec![ + // since file doesn't have "a" it will be filled with nulls + Field::new("b", DataType::Float64, true), + ]); + + let adapter = DefaultSchemaAdapterFactory::from_schema(Arc::new(table_schema)); + let (mapper, indices) = adapter.map_schema(&file_schema).unwrap(); + assert_eq!(indices, vec![0]); + + let file_batch = record_batch!(("b", Float64, vec![1.0, 2.0])).unwrap(); + + // Mapping fails because it tries to fill in a non-nullable column with nulls + let err = mapper.map_batch(file_batch).unwrap_err().to_string(); + assert!(err.contains("Invalid argument error: Column 'a' is declared as non-nullable but contains null values"), "{err}"); + } + #[derive(Debug)] struct TestSchemaAdapterFactory; diff --git a/datafusion/core/tests/macro_hygiene/mod.rs b/datafusion/core/tests/macro_hygiene/mod.rs index 72ac6e64fb0c..c35e46c0c558 100644 --- a/datafusion/core/tests/macro_hygiene/mod.rs +++ b/datafusion/core/tests/macro_hygiene/mod.rs @@ -37,3 +37,13 @@ mod plan_datafusion_err { plan_datafusion_err!("foo"); } } + +mod record_batch { + // NO other imports! + use datafusion_common::record_batch; + + #[test] + fn test_macro() { + record_batch!(("column_name", Int32, vec![1, 2, 3])).unwrap(); + } +} From 146f16a0c14f0fe65e2bd8b7226508f27ced3f13 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Sat, 26 Oct 2024 09:16:01 -0700 Subject: [PATCH 27/27] Move filtered SMJ Left Anti filtered join out of `join_partial` phase (#13111) * Move filtered SMJ Left Anti filtered join out of `join_partial` phase --- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 6 +- .../src/joins/sort_merge_join.rs | 245 ++++++++++- .../test_files/sort_merge_join.slt | 383 +++++++++--------- 3 files changed, 414 insertions(+), 220 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index ca2c2bf4e438..44d34b674bbb 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -41,6 +41,7 @@ use datafusion::physical_plan::joins::{ }; use datafusion::physical_plan::memory::MemoryExec; +use crate::fuzz_cases::join_fuzz::JoinTestType::NljHj; use datafusion::prelude::{SessionConfig, SessionContext}; use test_utils::stagger_batch_with_seed; @@ -223,9 +224,6 @@ async fn test_anti_join_1k() { } #[tokio::test] -// flaky for HjSmj case, giving 1 rows difference sometimes -// https://github.com/apache/datafusion/issues/11555 -#[ignore] async fn test_anti_join_1k_filtered() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -233,7 +231,7 @@ async fn test_anti_join_1k_filtered() { JoinType::LeftAnti, Some(Box::new(col_lt_col_filter)), ) - .run_test(&[JoinTestType::NljHj], false) + .run_test(&[JoinTestType::HjSmj, NljHj], false) .await } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index d5134855440a..7b7b7462f7e4 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -802,6 +802,32 @@ fn get_corrected_filter_mask( Some(corrected_mask.finish()) } + JoinType::LeftAnti => { + for i in 0..row_indices_length { + let last_index = + last_index_for_row(i, row_indices, batch_ids, row_indices_length); + + if filter_mask.value(i) { + seen_true = true; + } + + if last_index { + if !seen_true { + corrected_mask.append_value(true); + } else { + corrected_mask.append_null(); + } + + seen_true = false; + } else { + corrected_mask.append_null(); + } + } + + let null_matched = expected_size - corrected_mask.len(); + corrected_mask.extend(vec![Some(true); null_matched]); + Some(corrected_mask.finish()) + } // Only outer joins needs to keep track of processed rows and apply corrected filter mask _ => None, } @@ -835,15 +861,18 @@ impl Stream for SMJStream { JoinType::Left | JoinType::LeftSemi | JoinType::Right + | JoinType::LeftAnti ) { self.freeze_all()?; if !self.output_record_batches.batches.is_empty() - && self.buffered_data.scanning_finished() { - let out_batch = self.filter_joined_batch()?; - return Poll::Ready(Some(Ok(out_batch))); + let out_filtered_batch = + self.filter_joined_batch()?; + return Poll::Ready(Some(Ok( + out_filtered_batch, + ))); } } @@ -907,15 +936,17 @@ impl Stream for SMJStream { // because target output batch size can be hit in the middle of // filtering causing the filtering to be incomplete and causing // correctness issues - let record_batch = if !(self.filter.is_some() + if self.filter.is_some() && matches!( self.join_type, - JoinType::Left | JoinType::LeftSemi | JoinType::Right - )) { - record_batch - } else { + JoinType::Left + | JoinType::LeftSemi + | JoinType::Right + | JoinType::LeftAnti + ) + { continue; - }; + } return Poll::Ready(Some(Ok(record_batch))); } @@ -929,7 +960,10 @@ impl Stream for SMJStream { if self.filter.is_some() && matches!( self.join_type, - JoinType::Left | JoinType::LeftSemi | JoinType::Right + JoinType::Left + | JoinType::LeftSemi + | JoinType::Right + | JoinType::LeftAnti ) { let out = self.filter_joined_batch()?; @@ -1273,11 +1307,7 @@ impl SMJStream { }; if matches!(self.join_type, JoinType::LeftAnti) && self.filter.is_some() { - join_streamed = !self - .streamed_batch - .join_filter_matched_idxs - .contains(&(self.streamed_batch.idx as u64)) - && !self.streamed_joined; + join_streamed = !self.streamed_joined; join_buffered = join_streamed; } } @@ -1519,7 +1549,10 @@ impl SMJStream { // Push the filtered batch which contains rows passing join filter to the output if matches!( self.join_type, - JoinType::Left | JoinType::LeftSemi | JoinType::Right + JoinType::Left + | JoinType::LeftSemi + | JoinType::Right + | JoinType::LeftAnti ) { self.output_record_batches .batches @@ -1654,7 +1687,10 @@ impl SMJStream { if !(self.filter.is_some() && matches!( self.join_type, - JoinType::Left | JoinType::LeftSemi | JoinType::Right + JoinType::Left + | JoinType::LeftSemi + | JoinType::Right + | JoinType::LeftAnti )) { self.output_record_batches.batches.clear(); @@ -1727,7 +1763,7 @@ impl SMJStream { &self.schema, &[filtered_record_batch, null_joined_streamed_batch], )?; - } else if matches!(self.join_type, JoinType::LeftSemi) { + } else if matches!(self.join_type, JoinType::LeftSemi | JoinType::LeftAnti) { let output_column_indices = (0..streamed_columns_length).collect::>(); filtered_record_batch = filtered_record_batch.project(&output_column_indices)?; @@ -3349,6 +3385,7 @@ mod tests { batch_ids: vec![], }; + // Insert already prejoined non-filtered rows batches.batches.push(RecordBatch::try_new( Arc::clone(&schema), vec![ @@ -3835,6 +3872,178 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_left_anti_join_filtered_mask() -> Result<()> { + let mut joined_batches = build_joined_record_batches()?; + let schema = joined_batches.batches.first().unwrap().schema(); + + let output = concat_batches(&schema, &joined_batches.batches)?; + let out_mask = joined_batches.filter_mask.finish(); + let out_indices = joined_batches.row_indices.finish(); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0]), + &[0usize], + &BooleanArray::from(vec![true]), + 1 + ) + .unwrap(), + BooleanArray::from(vec![None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0]), + &[0usize], + &BooleanArray::from(vec![false]), + 1 + ) + .unwrap(), + BooleanArray::from(vec![Some(true)]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0]), + &[0usize; 2], + &BooleanArray::from(vec![true, true]), + 2 + ) + .unwrap(), + BooleanArray::from(vec![None, None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0, 0]), + &[0usize; 3], + &BooleanArray::from(vec![true, true, true]), + 3 + ) + .unwrap(), + BooleanArray::from(vec![None, None, None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0, 0]), + &[0usize; 3], + &BooleanArray::from(vec![true, false, true]), + 3 + ) + .unwrap(), + BooleanArray::from(vec![None, None, None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0, 0]), + &[0usize; 3], + &BooleanArray::from(vec![false, false, true]), + 3 + ) + .unwrap(), + BooleanArray::from(vec![None, None, None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0, 0]), + &[0usize; 3], + &BooleanArray::from(vec![false, true, true]), + 3 + ) + .unwrap(), + BooleanArray::from(vec![None, None, None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0, 0]), + &[0usize; 3], + &BooleanArray::from(vec![false, false, false]), + 3 + ) + .unwrap(), + BooleanArray::from(vec![None, None, Some(true)]) + ); + + let corrected_mask = get_corrected_filter_mask( + LeftAnti, + &out_indices, + &joined_batches.batch_ids, + &out_mask, + output.num_rows(), + ) + .unwrap(); + + assert_eq!( + corrected_mask, + BooleanArray::from(vec![ + None, + None, + None, + None, + None, + Some(true), + None, + Some(true) + ]) + ); + + let filtered_rb = filter_record_batch(&output, &corrected_mask)?; + + assert_batches_eq!( + &[ + "+---+----+---+----+", + "| a | b | x | y |", + "+---+----+---+----+", + "| 1 | 13 | 1 | 12 |", + "| 1 | 14 | 1 | 11 |", + "+---+----+---+----+", + ], + &[filtered_rb] + ); + + // output null rows + let null_mask = arrow::compute::not(&corrected_mask)?; + assert_eq!( + null_mask, + BooleanArray::from(vec![ + None, + None, + None, + None, + None, + Some(false), + None, + Some(false), + ]) + ); + + let null_joined_batch = filter_record_batch(&output, &null_mask)?; + + assert_batches_eq!( + &[ + "+---+---+---+---+", + "| a | b | x | y |", + "+---+---+---+---+", + "+---+---+---+---+", + ], + &[null_joined_batch] + ); + Ok(()) + } + /// Returns the column names on the schema fn columns(schema: &Schema) -> Vec { schema.fields().iter().map(|f| f.name().clone()).collect() diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index 051cc6dce3d4..f4cc888d6b8e 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -407,214 +407,201 @@ select t1.* from t1 where exists (select 1 from t2 where t2.a = t1.a and t2.b != statement ok set datafusion.execution.batch_size = 10; -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 13 c union all -# select 11 a, 14 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- -#11 12 - -# Uncomment when filtered LEFTANTI moved -#query III -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b, 1 c union all -# select 11 a, 13 b, 2 c), -#t2 as ( -# select 11 a, 12 b, 3 c union all -# select 11 a, 14 b, 4 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) -#) order by 1, 2; -#---- -#11 12 1 -#11 13 2 - -# Uncomment when filtered LEFTANTI moved -#query III -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b, 1 c union all -# select 11 a, 13 b, 2 c), -#t2 as ( -# select 11 a, 12 b, 3 c where false -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) -#) order by 1, 2; -#---- -#11 12 1 -#11 13 2 - -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 13 c union all -# select 11 a, 14 c union all -# select 11 a, 15 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- -#11 12 - -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 11 c union all -# select 11 a, 14 c union all -# select 11 a, 15 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 13 c union all + select 11 a, 14 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- +11 12 -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 12 c union all -# select 11 a, 11 c union all -# select 11 a, 15 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- +query III +select * from ( +with +t1 as ( + select 11 a, 12 b, 1 c union all + select 11 a, 13 b, 2 c), +t2 as ( + select 11 a, 12 b, 3 c union all + select 11 a, 14 b, 4 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) +) order by 1, 2; +---- +11 12 1 +11 13 2 +query III +select * from ( +with +t1 as ( + select 11 a, 12 b, 1 c union all + select 11 a, 13 b, 2 c), +t2 as ( + select 11 a, 12 b, 3 c where false + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) +) order by 1, 2; +---- +11 12 1 +11 13 2 -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 12 c union all -# select 11 a, 14 c union all -# select 11 a, 11 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 13 c union all + select 11 a, 14 c union all + select 11 a, 15 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- +11 12 + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 11 c union all + select 11 a, 14 c union all + select 11 a, 15 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 12 c union all + select 11 a, 11 c union all + select 11 a, 15 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- + + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 12 c union all + select 11 a, 14 c union all + select 11 a, 11 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- # Test LEFT ANTI with cross batch data distribution statement ok set datafusion.execution.batch_size = 1; -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 13 c union all -# select 11 a, 14 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- -#11 12 - -# Uncomment when filtered LEFTANTI moved -#query III -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b, 1 c union all -# select 11 a, 13 b, 2 c), -#t2 as ( -# select 11 a, 12 b, 3 c union all -# select 11 a, 14 b, 4 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) -#) order by 1, 2; -#---- -#11 12 1 -#11 13 2 - -# Uncomment when filtered LEFTANTI moved -#query III -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b, 1 c union all -# select 11 a, 13 b, 2 c), -#t2 as ( -# select 11 a, 12 b, 3 c where false -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) -#) order by 1, 2; -#---- -#11 12 1 -#11 13 2 - -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 13 c union all -# select 11 a, 14 c union all -# select 11 a, 15 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- -#11 12 - -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 12 c union all -# select 11 a, 11 c union all -# select 11 a, 15 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 13 c union all + select 11 a, 14 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- +11 12 -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 12 c union all -# select 11 a, 14 c union all -# select 11 a, 11 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- +query III +select * from ( +with +t1 as ( + select 11 a, 12 b, 1 c union all + select 11 a, 13 b, 2 c), +t2 as ( + select 11 a, 12 b, 3 c union all + select 11 a, 14 b, 4 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) +) order by 1, 2; +---- +11 12 1 +11 13 2 + +query III +select * from ( +with +t1 as ( + select 11 a, 12 b, 1 c union all + select 11 a, 13 b, 2 c), +t2 as ( + select 11 a, 12 b, 3 c where false + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) +) order by 1, 2; +---- +11 12 1 +11 13 2 + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 13 c union all + select 11 a, 14 c union all + select 11 a, 15 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- +11 12 + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 12 c union all + select 11 a, 11 c union all + select 11 a, 15 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 12 c union all + select 11 a, 14 c union all + select 11 a, 11 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- query IIII select * from (