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] 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