From afc605905098a7d28a86994b9c4b465a98e61817 Mon Sep 17 00:00:00 2001 From: pavle-martinovic_data Date: Tue, 5 Aug 2025 16:11:50 +0200 Subject: [PATCH 01/10] [SPARK-53127] Enable LIMIT ALL to override recursion row limit --- .../spark/sql/catalyst/dsl/package.scala | 2 + .../sql/catalyst/optimizer/Optimizer.scala | 17 +++++ .../sql/catalyst/parser/AstBuilder.scala | 17 ++++- .../plans/logical/basicLogicalOperators.scala | 9 +++ .../optimizer/LimitPushdownSuite.scala | 34 ++++++++- .../spark/sql/execution/UnionLoopExec.scala | 6 +- .../analyzer-results/cte-recursion.sql.out | 26 +++++++ .../sql-tests/analyzer-results/limit.sql.out | 9 +-- .../analyzer-results/pipe-operators.sql.out | 9 +-- .../sql-tests/inputs/cte-recursion.sql | 8 +++ .../sql-tests/results/cte-recursion.sql.out | 72 +++++++++++++++++++ 11 files changed, 195 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index b4168820ffe54..1d7cf5455e57b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -405,6 +405,8 @@ package object dsl extends SQLConfHelper { def globalLimit(limitExpr: Expression): LogicalPlan = GlobalLimit(limitExpr, logicalPlan) + def limitAll(): LogicalPlan = LimitAll(logicalPlan) + def offset(offsetExpr: Expression): LogicalPlan = Offset(offsetExpr, logicalPlan) def join( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 4a828375f9d07..2fe728ac3ce17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -882,6 +882,23 @@ object LimitPushDown extends Rule[LogicalPlan] { LocalLimit(le, udf.copy(child = maybePushLocalLimit(le, udf.child))) case LocalLimit(le, p @ Project(_, udf: ArrowEvalPython)) => LocalLimit(le, p.copy(child = udf.copy(child = maybePushLocalLimit(le, udf.child)))) + + // Logic for pushing down LimitAll node. + // This node is no-op for any node that isn't UnionLoop. For UnionLoop it is used to signify an + // infinite recursion. + case LimitAll(ul @ UnionLoop(_, _, _, _, None, _)) => + ul.copy(limit = Some(-1)) + // If a limit node is present, the LimitAll shouldn't get pushed down, as the lowest limit node + // above a UnionLoop should only be considered. + case LimitAll(limit: GlobalLimit) => + limit + case LimitAll(limit: LocalLimit) => + limit + case LimitAll(ul @ UnionLoop(_, _, _, _, Some(_), _)) => + ul + // Propagate LimitAll to all children. + case LimitAll(child) => + child.withNewChildren(child.children.map(LimitAll(_))) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index b03a8811da308..19cf5ece98868 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1346,15 +1346,26 @@ class AstBuilder extends DataTypeAstBuilder } // LIMIT - // - LIMIT ALL is the same as omitting the LIMIT clause - withOffset.optional(limit) { + // LIMIT ALL creates LimitAll node which can be used for infinite recursions in recursive CTEs. + if (ctx.ALL() != null) { if (forPipeOperators && clause.nonEmpty && clause != PipeOperators.offsetClause) { throw QueryParsingErrors.multipleQueryResultClausesWithPipeOperatorsUnsupportedError( ctx, clause, PipeOperators.limitClause) } clause = PipeOperators.limitClause - Limit(typedVisit(limit), withOffset) + LimitAll(withOffset) + } else { + withOffset.optional(limit) { + if (forPipeOperators && clause.nonEmpty && clause != PipeOperators.offsetClause) { + throw QueryParsingErrors.multipleQueryResultClausesWithPipeOperatorsUnsupportedError( + ctx, clause, PipeOperators.limitClause) + } + clause = PipeOperators.limitClause + Limit(typedVisit(limit), withOffset) + } } + + } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 515424db63567..8295f13bb6545 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -1651,6 +1651,15 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends OrderPr copy(child = newChild) } +case class LimitAll(child: LogicalPlan) extends OrderPreservingUnaryNode { + override def output: Seq[Attribute] = child.output + + final override val nodePatterns: Seq[TreePattern] = Seq(LIMIT) + + override protected def withNewChildInternal(newChild: LogicalPlan): LimitAll = + copy(child = newChild) +} + object OffsetAndLimit { def unapply(p: GlobalLimit): Option[(Int, Int, LogicalPlan)] = { p match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala index 2dcab5cfd29c4..2dec2908b18c6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Add, GenericInternalRow} +import org.apache.spark.sql.catalyst.expressions.{Add, ExprId, GenericInternalRow} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -356,4 +356,36 @@ class LimitPushdownSuite extends PlanTest { comparePlans(Optimize.execute(originalQuery2), originalQuery2) } } + + test("LimitAll node gets eliminated without UnionLoop") { + val originalQuery = x.join(y.limitAll(), RightOuter).limitAll() + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = x.join(y, RightOuter).analyze + comparePlans(optimized, correctAnswer) + } + + test("LimitAll node gets pushed into UnionLoop") { + val originalQuery = UnionLoop(0, x, y, Seq.empty).limitAll() + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = UnionLoop(0, x, y, Seq.empty, Some(-1)).analyze + comparePlans(optimized, correctAnswer) + } + + test("LimitAll node gets pushed into UnionLoop through other nodes") { + val originalQuery = Union(x, UnionLoop(0, x, y, Seq(ExprId(1), ExprId(2), ExprId(3)) + )).limitAll() + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = Union(x, UnionLoop(0, x, y, Seq(ExprId(1), ExprId(2), ExprId(3)), + Some(-1))).analyze + comparePlans(optimized, correctAnswer) + } + + test("LimitAll node gets eliminated before UnionLoop due to previous Limit") { + val originalQuery = Union(x, UnionLoop(0, x, y, Seq(ExprId(1), ExprId(2), ExprId(3)) + ).localLimit(5)).limitAll() + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = Union(x, UnionLoop(0, x, y, Seq(ExprId(1), ExprId(2), ExprId(3)), + Some(5))).analyze + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala index 80b884d240972..664ba4bda166b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala @@ -106,7 +106,7 @@ case class UnionLoopExec( private def executeAndCacheAndCount(plan: LogicalPlan, currentLimit: Int) = { // In case limit is defined, we create a (local) limit node above the plan and execute // the newly created plan. - val planWithLimit = if (limit.isDefined) { + val planWithLimit = if (limit.isDefined && limit.get != -1) { LocalLimit(Literal(currentLimit), plan) } else { plan @@ -167,6 +167,8 @@ case class UnionLoopExec( // the user knows they aren't getting all the rows they requested. var currentLimit = limit.getOrElse(rowLimit) + val unlimitedRecursion = currentLimit == -1 + val userSpecifiedLimit = limit.isDefined val unionChildren = mutable.ArrayBuffer.empty[LogicalPlan] @@ -237,7 +239,7 @@ case class UnionLoopExec( unionChildren += prevPlan - if (rowLimit != -1) { + if (!unlimitedRecursion) { currentLimit -= prevCount.toInt if (currentLimit <= 0) { if (userSpecifiedLimit) { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out index 6bb0b27ca9ea1..b6d407a5cfd6a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out @@ -293,6 +293,32 @@ DROP VIEW ZeroAndOne DropTempViewCommand ZeroAndOne +-- !query +WITH RECURSIVE t(n) MAX RECURSION LEVEL 100 AS ( + SELECT 1 + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) +SELECT * FROM t LIMIT ALL +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, 100 +: +- SubqueryAlias t +: +- Project [1#x AS n#x] +: +- UnionLoop xxxx, 100 +: :- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Project [(n#x + 1) AS (n + 1)#x] +: +- Filter (n#x < 60) +: +- SubqueryAlias t +: +- Project [1#x AS n#x] +: +- UnionLoopRef xxxx, [1#x], false ++- LimitAll + +- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + -- !query WITH RECURSIVE r(level) AS ( VALUES 0 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out index e92dcfbc069a7..984c7004c1f5b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out @@ -197,7 +197,8 @@ Project [id#xL] -- !query SELECT * FROM testdata WHERE key < 3 LIMIT ALL -- !query analysis -Project [key#x, value#x] -+- Filter (key#x < 3) - +- SubqueryAlias spark_catalog.default.testdata - +- Relation spark_catalog.default.testdata[key#x,value#x] parquet +LimitAll ++- Project [key#x, value#x] + +- Filter (key#x < 3) + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index dda0722e21d75..5c3e2444f9fa1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -3112,10 +3112,11 @@ Offset 1 table t |> limit all offset 0 -- !query analysis -Offset 0 -+- PipeOperator - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv +LimitAll ++- Offset 0 + +- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql index 05ddd7fb89fab..9e0584ec93dce 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql @@ -97,6 +97,14 @@ SELECT * FROM t LIMIT 60; DROP VIEW ZeroAndOne; +-- limited recursion allowed to stop from failing by putting LIMIT ALL +WITH RECURSIVE t(n) MAX RECURSION LEVEL 100 AS ( + SELECT 1 + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) +SELECT * FROM t LIMIT ALL; + -- terminate recursion with LIMIT WITH RECURSIVE r(level) AS ( VALUES 0 diff --git a/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out index 310f313cd2a38..fc5a6245c634b 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out @@ -333,6 +333,78 @@ struct<> +-- !query +WITH RECURSIVE t(n) MAX RECURSION LEVEL 100 AS ( + SELECT 1 + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) +SELECT * FROM t LIMIT ALL +-- !query schema +struct +-- !query output +1 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +2 +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +3 +30 +31 +32 +33 +34 +35 +36 +37 +38 +39 +4 +40 +41 +42 +43 +44 +45 +46 +47 +48 +49 +5 +50 +51 +52 +53 +54 +55 +56 +57 +58 +59 +6 +60 +7 +8 +9 + + -- !query WITH RECURSIVE r(level) AS ( VALUES 0 From a4198ddd83cd86b43ebf759d7a1c72a64f189b55 Mon Sep 17 00:00:00 2001 From: pavle-martinovic_data Date: Mon, 11 Aug 2025 17:19:27 +0200 Subject: [PATCH 02/10] replace LimitAll node with custom expr --- .../sql/catalyst/analysis/CheckAnalysis.scala | 1 + .../catalyst/expressions/LimitAllExpr.scala | 35 +++++++++++++++++++ .../sql/catalyst/optimizer/Optimizer.scala | 16 ++++----- .../plans/logical/basicLogicalOperators.scala | 16 +++++---- 4 files changed, 54 insertions(+), 14 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 014b8ebece046..7d8b6da917ca3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -98,6 +98,7 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString private def checkLimitLikeClause(name: String, limitExpr: Expression): Unit = { limitExpr match { + case e if e == LimitAllExpr => // Limit all expression is a special case which is allowed. case e if !e.foldable => limitExpr.failAnalysis( errorClass = "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", messageParameters = Map( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala new file mode 100644 index 0000000000000..493d6c61d88f5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala @@ -0,0 +1,35 @@ +/* + * 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. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.types.{DataType, IntegerType} + +case object LimitAllExpr extends LeafExpression { + override def nullable: Boolean = false + override def dataType: DataType = IntegerType + + override def eval(input: InternalRow): Any = + throw new UnsupportedOperationException("LimitAllExpr should not be evaluated directly.") + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + throw new UnsupportedOperationException("LimitAllExpr should not be codegened.") + + override def prettyName: String = "limit_all" +} \ No newline at end of file diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 2fe728ac3ce17..c8cb3359a8a41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -833,7 +833,7 @@ object LimitPushDown extends Rule[LogicalPlan] { // Note: right now Union means UNION ALL, which does not de-duplicate rows, so it is safe to // pushdown Limit through it. Once we add UNION DISTINCT, however, we will not be able to // pushdown Limit. - case LocalLimit(exp, u: Union) => + case LocalLimit(exp, u: Union) if exp != LimitAllExpr => LocalLimit(exp, u.copy(children = u.children.map(maybePushLocalLimit(exp, _)))) // If limit node is present, we should propagate it down to UnionLoop, so that it is later @@ -855,10 +855,10 @@ object LimitPushDown extends Rule[LogicalPlan] { // not eventually introduce limits on both sides if it is applied multiple times. Therefore: // - If one side is already limited, stack another limit on top if the new limit is smaller. // The redundant limit will be collapsed by the CombineLimits rule. - case LocalLimit(exp, join: Join) => + case LocalLimit(exp, join: Join) if exp != LimitAllExpr => LocalLimit(exp, pushLocalLimitThroughJoin(exp, join)) // There is a Project between LocalLimit and Join if they do not have the same output. - case LocalLimit(exp, project @ Project(_, join: Join)) => + case LocalLimit(exp, project @ Project(_, join: Join)) if exp != LimitAllExpr => LocalLimit(exp, project.copy(child = pushLocalLimitThroughJoin(exp, join))) // Push down limit 1 through Aggregate and turn Aggregate into Project if it is group only. case Limit(le @ IntegerLiteral(1), a: Aggregate) if a.groupOnly => @@ -868,19 +868,19 @@ object LimitPushDown extends Rule[LogicalPlan] { val newAgg = EliminateSorts(a.copy(child = LocalLimit(le, a.child))).asInstanceOf[Aggregate] Limit(le, p.copy(child = Project(newAgg.aggregateExpressions, newAgg.child))) // Merge offset value and limit value into LocalLimit and pushes down LocalLimit through Offset. - case LocalLimit(le, Offset(oe, grandChild)) => + case LocalLimit(le, Offset(oe, grandChild)) if le != LimitAllExpr => Offset(oe, LocalLimit(Add(le, oe), grandChild)) // Push down local limit 1 if join type is LeftSemiOrAnti and join condition is empty. case j @ Join(_, right, LeftSemiOrAnti(_), None, _) if !right.maxRows.exists(_ <= 1) => j.copy(right = maybePushLocalLimit(Literal(1, IntegerType), right)) // Push down limits through Python UDFs. - case LocalLimit(le, udf: BatchEvalPython) => + case LocalLimit(le, udf: BatchEvalPython) if le != LimitAllExpr => LocalLimit(le, udf.copy(child = maybePushLocalLimit(le, udf.child))) - case LocalLimit(le, p @ Project(_, udf: BatchEvalPython)) => + case LocalLimit(le, p @ Project(_, udf: BatchEvalPython)) if le != LimitAllExpr => LocalLimit(le, p.copy(child = udf.copy(child = maybePushLocalLimit(le, udf.child)))) - case LocalLimit(le, udf: ArrowEvalPython) => + case LocalLimit(le, udf: ArrowEvalPython) if le != LimitAllExpr => LocalLimit(le, udf.copy(child = maybePushLocalLimit(le, udf.child))) - case LocalLimit(le, p @ Project(_, udf: ArrowEvalPython)) => + case LocalLimit(le, p @ Project(_, udf: ArrowEvalPython)) if le != LimitAllExpr => LocalLimit(le, p.copy(child = udf.copy(child = maybePushLocalLimit(le, udf.child)))) // Logic for pushing down LimitAll node. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 8295f13bb6545..b718518d36054 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -1651,13 +1651,17 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends OrderPr copy(child = newChild) } -case class LimitAll(child: LogicalPlan) extends OrderPreservingUnaryNode { - override def output: Seq[Attribute] = child.output - - final override val nodePatterns: Seq[TreePattern] = Seq(LIMIT) +object LimitAll { + def apply(child: LogicalPlan): UnaryNode = { + LocalLimit(LimitAllExpr, child) + } - override protected def withNewChildInternal(newChild: LogicalPlan): LimitAll = - copy(child = newChild) + def unapply(p: LocalLimit): Option[LogicalPlan] = { + p.limitExpr match { + case LimitAllExpr => Some(p.child) + case _ => None + } + } } object OffsetAndLimit { From fb22de5d866d92621c0b73238726747fb284b73b Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 13 Aug 2025 08:39:18 +0900 Subject: [PATCH 03/10] Update sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala --- .../apache/spark/sql/catalyst/expressions/LimitAllExpr.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala index 493d6c61d88f5..15e4e207226cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala @@ -32,4 +32,4 @@ case object LimitAllExpr extends LeafExpression { throw new UnsupportedOperationException("LimitAllExpr should not be codegened.") override def prettyName: String = "limit_all" -} \ No newline at end of file +} From 22bcc52eecddfd20539cc26f1fd225fe5adbe50c Mon Sep 17 00:00:00 2001 From: pavle-martinovic_data Date: Thu, 14 Aug 2025 18:30:24 +0200 Subject: [PATCH 04/10] change to unevaluable, fix limit all offset bug, regen golden files --- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 2 +- .../spark/sql/catalyst/expressions/LimitAllExpr.scala | 11 ++--------- .../sql-tests/analyzer-results/cte-recursion.sql.out | 2 +- .../sql-tests/analyzer-results/limit.sql.out | 2 +- .../sql-tests/analyzer-results/pipe-operators.sql.out | 2 +- 5 files changed, 6 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 7d8b6da917ca3..60beeea3e530c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -635,7 +635,7 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString case LocalLimit(limitExpr, child) => checkLimitLikeClause("limit", limitExpr) child match { - case Offset(offsetExpr, _) => + case Offset(offsetExpr, _) if limitExpr != LimitAllExpr => val limit = limitExpr.eval().asInstanceOf[Int] val offset = offsetExpr.eval().asInstanceOf[Int] if (Int.MaxValue - limit < offset) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala index 15e4e207226cf..3b34681d695f7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala @@ -17,19 +17,12 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.types.{DataType, IntegerType} -case object LimitAllExpr extends LeafExpression { +case object LimitAllExpr extends LeafExpression with Unevaluable { override def nullable: Boolean = false - override def dataType: DataType = IntegerType - - override def eval(input: InternalRow): Any = - throw new UnsupportedOperationException("LimitAllExpr should not be evaluated directly.") - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = - throw new UnsupportedOperationException("LimitAllExpr should not be codegened.") + override def dataType: DataType = IntegerType override def prettyName: String = "limit_all" } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out index b6d407a5cfd6a..787a108781df8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out @@ -313,7 +313,7 @@ WithCTE : +- SubqueryAlias t : +- Project [1#x AS n#x] : +- UnionLoopRef xxxx, [1#x], false -+- LimitAll ++- LocalLimit limit_all() +- Project [n#x] +- SubqueryAlias t +- CTERelationRef xxxx, true, [n#x], false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out index 984c7004c1f5b..337c8f31bceb8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out @@ -197,7 +197,7 @@ Project [id#xL] -- !query SELECT * FROM testdata WHERE key < 3 LIMIT ALL -- !query analysis -LimitAll +LocalLimit limit_all() +- Project [key#x, value#x] +- Filter (key#x < 3) +- SubqueryAlias spark_catalog.default.testdata diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index 5c3e2444f9fa1..5f7c9b379b7a4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -3112,7 +3112,7 @@ Offset 1 table t |> limit all offset 0 -- !query analysis -LimitAll +LocalLimit limit_all() +- Offset 0 +- PipeOperator +- SubqueryAlias spark_catalog.default.t From 91f2d0da515b18f23860657dbbc05f9d7495d55d Mon Sep 17 00:00:00 2001 From: pavle-martinovic_data Date: Fri, 29 Aug 2025 18:12:54 +0200 Subject: [PATCH 05/10] replace limitAll to get resolved in analysis --- .../sql/catalyst/analysis/Analyzer.scala | 20 +++++++++++ .../sql/catalyst/analysis/CheckAnalysis.scala | 3 +- .../catalyst/analysis/ResolveWithCTE.scala | 3 +- .../catalyst/expressions/LimitAllExpr.scala | 28 ---------------- .../sql/catalyst/optimizer/InlineCTE.scala | 21 +++++++++--- .../sql/catalyst/optimizer/Optimizer.scala | 33 +++++-------------- ...wnPredicatesAndPruneColumnsForCTEDef.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 16 ++++----- .../catalyst/plans/logical/cteOperators.scala | 9 ++--- .../analyzer-results/cte-recursion.sql.out | 7 ++-- .../sql-tests/analyzer-results/limit.sql.out | 9 +++-- .../analyzer-results/pipe-operators.sql.out | 9 +++-- .../apache/spark/sql/ParametersSuite.scala | 2 +- 13 files changed, 72 insertions(+), 90 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 1896a1c7ac279..5c290824544ef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -422,6 +422,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor WindowsSubstitution, EliminateUnions, EliminateLazyExpression), + Batch("Apply Limit All", Once, ApplyLimitAll), Batch("Disable Hints", Once, new ResolveHints.DisableHints), Batch("Hints", fixedPoint, @@ -4281,6 +4282,25 @@ object RemoveTempResolvedColumn extends Rule[LogicalPlan] { } } +object ApplyLimitAll extends Rule[LogicalPlan] { + def applyLimitAllToPlan(plan: LogicalPlan, isInLimitAll: Boolean = false): LogicalPlan = { + plan match { + case la: LimitAll => + applyLimitAllToPlan(la.child, isInLimitAll = true) + case cteRef: CTERelationRef if isInLimitAll => + cteRef.copy(isUnlimitedRecursion = Some(true)) + case other => + other.withNewChildren(other.children + .map(child => applyLimitAllToPlan(child, isInLimitAll))) + } + } + + def apply(plan: LogicalPlan): LogicalPlan = { + val ret = applyLimitAllToPlan(plan) + ret + } +} + /** * Rule that's used to handle `UnresolvedHaving` nodes with resolved `condition` and `child`. * It's placed outside the main batch to avoid conflicts with other rules that resolve diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index f9aaea45565cf..2ff842553bee6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -99,7 +99,6 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString private def checkLimitLikeClause(name: String, limitExpr: Expression): Unit = { limitExpr match { - case e if e == LimitAllExpr => // Limit all expression is a special case which is allowed. case e if !e.foldable => limitExpr.failAnalysis( errorClass = "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", messageParameters = Map( @@ -636,7 +635,7 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString case LocalLimit(limitExpr, child) => checkLimitLikeClause("limit", limitExpr) child match { - case Offset(offsetExpr, _) if limitExpr != LimitAllExpr => + case Offset(offsetExpr, _) => val limit = limitExpr.eval().asInstanceOf[Int] val offset = offsetExpr.eval().asInstanceOf[Int] if (Int.MaxValue - limit < offset) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala index 0f6fcec467ce1..5bbb4ddecc7f0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala @@ -286,7 +286,8 @@ object ResolveWithCTE extends Rule[LogicalPlan] { cteDefMap.get(ref.cteId).map { cteDef => // cteDef is certainly resolved, otherwise it would not have been in the map. CTERelationRef( - cteDef.id, cteDef.resolved, cteDef.output, cteDef.isStreaming, maxRows = cteDef.maxRows) + cteDef.id, cteDef.resolved, cteDef.output, cteDef.isStreaming, maxRows = cteDef.maxRows, + isUnlimitedRecursion = ref.isUnlimitedRecursion) }.getOrElse { ref } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala deleted file mode 100644 index 3b34681d695f7..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/LimitAllExpr.scala +++ /dev/null @@ -1,28 +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. - */ - -package org.apache.spark.sql.catalyst.expressions - -import org.apache.spark.sql.types.{DataType, IntegerType} - -case object LimitAllExpr extends LeafExpression with Unevaluable { - override def nullable: Boolean = false - - override def dataType: DataType = IntegerType - - override def prettyName: String = "limit_all" -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala index ad1a1a99b8257..91f0b15bbd944 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.analysis.DeduplicateRelations import org.apache.spark.sql.catalyst.expressions.{Alias, OuterReference, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, Join, JoinHint, LogicalPlan, Project, Subquery, WithCTE} +import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, Join, JoinHint, LogicalPlan, Project, Subquery, SubqueryAlias, UnionLoop, WithCTE} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{CTE, PLAN_EXPRESSION} @@ -189,14 +189,27 @@ case class InlineCTE( case ref: CTERelationRef => val refInfo = cteMap(ref.cteId) + val cteBody = refInfo.cteDef.child match { + case sa @ SubqueryAlias(_, ul: UnionLoop) + if ul.id == ref.cteId && ref.isUnlimitedRecursion.isDefined => + sa.copy(child = ul.copy(limit = Some(-1))) + case sa @ SubqueryAlias(_, p @ Project(_, ul: UnionLoop)) + if ul.id == ref.cteId && ref.isUnlimitedRecursion.isDefined => + sa.copy(child = p.copy(child = ul.copy(limit = Some(-1)))) + case p @ Project(_, ul: UnionLoop) + if ul.id == ref.cteId && ref.isUnlimitedRecursion.isDefined => + p.copy(child = ul.copy(limit = Some(-1))) + case body => + body + } if (refInfo.shouldInline) { if (ref.outputSet == refInfo.cteDef.outputSet) { - refInfo.cteDef.child + cteBody } else { val ctePlan = DeduplicateRelations( Join( - refInfo.cteDef.child, - refInfo.cteDef.child, + cteBody, + cteBody, Inner, None, JoinHint(None, None) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 5d723b40fbdbd..ef505a0144113 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -878,7 +878,7 @@ object LimitPushDown extends Rule[LogicalPlan] { // Note: right now Union means UNION ALL, which does not de-duplicate rows, so it is safe to // pushdown Limit through it. Once we add UNION DISTINCT, however, we will not be able to // pushdown Limit. - case LocalLimit(exp, u: Union) if exp != LimitAllExpr => + case LocalLimit(exp, u: Union) => LocalLimit(exp, u.copy(children = u.children.map(maybePushLocalLimit(exp, _)))) // If limit node is present, we should propagate it down to UnionLoop, so that it is later @@ -900,10 +900,10 @@ object LimitPushDown extends Rule[LogicalPlan] { // not eventually introduce limits on both sides if it is applied multiple times. Therefore: // - If one side is already limited, stack another limit on top if the new limit is smaller. // The redundant limit will be collapsed by the CombineLimits rule. - case LocalLimit(exp, join: Join) if exp != LimitAllExpr => + case LocalLimit(exp, join: Join) => LocalLimit(exp, pushLocalLimitThroughJoin(exp, join)) // There is a Project between LocalLimit and Join if they do not have the same output. - case LocalLimit(exp, project @ Project(_, join: Join)) if exp != LimitAllExpr => + case LocalLimit(exp, project @ Project(_, join: Join)) => LocalLimit(exp, project.copy(child = pushLocalLimitThroughJoin(exp, join))) // Push down limit 1 through Aggregate and turn Aggregate into Project if it is group only. case Limit(le @ IntegerLiteral(1), a: Aggregate) if a.groupOnly => @@ -913,37 +913,20 @@ object LimitPushDown extends Rule[LogicalPlan] { val newAgg = EliminateSorts(a.copy(child = LocalLimit(le, a.child))).asInstanceOf[Aggregate] Limit(le, p.copy(child = Project(newAgg.aggregateExpressions, newAgg.child))) // Merge offset value and limit value into LocalLimit and pushes down LocalLimit through Offset. - case LocalLimit(le, Offset(oe, grandChild)) if le != LimitAllExpr => + case LocalLimit(le, Offset(oe, grandChild)) => Offset(oe, LocalLimit(Add(le, oe), grandChild)) // Push down local limit 1 if join type is LeftSemiOrAnti and join condition is empty. case j @ Join(_, right, LeftSemiOrAnti(_), None, _) if !right.maxRows.exists(_ <= 1) => j.copy(right = maybePushLocalLimit(Literal(1, IntegerType), right)) // Push down limits through Python UDFs. - case LocalLimit(le, udf: BatchEvalPython) if le != LimitAllExpr => + case LocalLimit(le, udf: BatchEvalPython) => LocalLimit(le, udf.copy(child = maybePushLocalLimit(le, udf.child))) - case LocalLimit(le, p @ Project(_, udf: BatchEvalPython)) if le != LimitAllExpr => + case LocalLimit(le, p @ Project(_, udf: BatchEvalPython)) => LocalLimit(le, p.copy(child = udf.copy(child = maybePushLocalLimit(le, udf.child)))) - case LocalLimit(le, udf: ArrowEvalPython) if le != LimitAllExpr => + case LocalLimit(le, udf: ArrowEvalPython) => LocalLimit(le, udf.copy(child = maybePushLocalLimit(le, udf.child))) - case LocalLimit(le, p @ Project(_, udf: ArrowEvalPython)) if le != LimitAllExpr => + case LocalLimit(le, p @ Project(_, udf: ArrowEvalPython)) => LocalLimit(le, p.copy(child = udf.copy(child = maybePushLocalLimit(le, udf.child)))) - - // Logic for pushing down LimitAll node. - // This node is no-op for any node that isn't UnionLoop. For UnionLoop it is used to signify an - // infinite recursion. - case LimitAll(ul @ UnionLoop(_, _, _, _, None, _)) => - ul.copy(limit = Some(-1)) - // If a limit node is present, the LimitAll shouldn't get pushed down, as the lowest limit node - // above a UnionLoop should only be considered. - case LimitAll(limit: GlobalLimit) => - limit - case LimitAll(limit: LocalLimit) => - limit - case LimitAll(ul @ UnionLoop(_, _, _, _, Some(_), _)) => - ul - // Propagate LimitAll to all children. - case LimitAll(child) => - child.withNewChildren(child.children.map(LimitAll(_))) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala index 32c2b569d9bba..421c47f12ddb3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala @@ -141,7 +141,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { cteDef } - case cteRef @ CTERelationRef(cteId, _, output, _, _, _, _) => + case cteRef @ CTERelationRef(cteId, _, output, _, _, _, _, _) => val (cteDef, _, _, newAttrSet) = cteMap(cteId) if (needsPruning(cteDef.child, newAttrSet)) { val indices = newAttrSet.toSeq.map(cteDef.output.indexOf) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index b81aee72448a8..8d51efea07a24 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -1659,17 +1659,13 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends OrderPr copy(child = newChild) } -object LimitAll { - def apply(child: LogicalPlan): UnaryNode = { - LocalLimit(LimitAllExpr, child) - } +case class LimitAll(child: LogicalPlan) extends OrderPreservingUnaryNode { + override def output: Seq[Attribute] = child.output - def unapply(p: LocalLimit): Option[LogicalPlan] = { - p.limitExpr match { - case LimitAllExpr => Some(p.child) - case _ => None - } - } + final override val nodePatterns: Seq[TreePattern] = Seq(LIMIT) + + override protected def withNewChildInternal(newChild: LogicalPlan): LimitAll = + copy(child = newChild) } object OffsetAndLimit { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala index c3c662c471f8f..59dab7b8f4db8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala @@ -128,7 +128,7 @@ case class CTERelationDef( override def output: Seq[Attribute] = if (resolved) child.output else Nil lazy val hasSelfReferenceAsCTERef: Boolean = child.collectFirstWithSubqueries { - case CTERelationRef(this.id, _, _, _, _, true, _) => true + case CTERelationRef(this.id, _, _, _, _, true, _, _) => true }.getOrElse(false) lazy val hasSelfReferenceInAnchor: Boolean = { val unionNode: Option[Union] = child match { @@ -144,7 +144,7 @@ case class CTERelationDef( } if (unionNode.isDefined) { unionNode.get.children.head.collectFirstWithSubqueries { - case CTERelationRef(this.id, _, _, _, _, true, _) => true + case CTERelationRef(this.id, _, _, _, _, true, _, _) => true }.getOrElse(false) } else { false @@ -160,7 +160,7 @@ case class CTERelationDef( } if (withCTENode.isDefined) { withCTENode.exists(_.cteDefs.exists(_.collectFirstWithSubqueries { - case CTERelationRef(this.id, _, _, _, _, true, _) => true + case CTERelationRef(this.id, _, _, _, _, true, _, _) => true }.isDefined)) } else { false @@ -194,7 +194,8 @@ case class CTERelationRef( override val isStreaming: Boolean, statsOpt: Option[Statistics] = None, recursive: Boolean = false, - override val maxRows: Option[Long] = None) extends LeafNode with MultiInstanceRelation { + override val maxRows: Option[Long] = None, + isUnlimitedRecursion: Option[Boolean] = None) extends LeafNode with MultiInstanceRelation { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out index faf4b9fe6be39..9b87d3ae542da 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out @@ -313,10 +313,9 @@ WithCTE : +- SubqueryAlias t : +- Project [1#x AS n#x] : +- UnionLoopRef xxxx, [1#x], false -+- LocalLimit limit_all() - +- Project [n#x] - +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false ++- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false, true -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out index 337c8f31bceb8..e92dcfbc069a7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out @@ -197,8 +197,7 @@ Project [id#xL] -- !query SELECT * FROM testdata WHERE key < 3 LIMIT ALL -- !query analysis -LocalLimit limit_all() -+- Project [key#x, value#x] - +- Filter (key#x < 3) - +- SubqueryAlias spark_catalog.default.testdata - +- Relation spark_catalog.default.testdata[key#x,value#x] parquet +Project [key#x, value#x] ++- Filter (key#x < 3) + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index 5f7c9b379b7a4..dda0722e21d75 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -3112,11 +3112,10 @@ Offset 1 table t |> limit all offset 0 -- !query analysis -LocalLimit limit_all() -+- Offset 0 - +- PipeOperator - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv +Offset 0 ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index 0c2380a2d90e3..22b1599cc35aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -792,7 +792,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession { | SELECT 1 | UNION ALL | SELECT n+1 FROM IDENTIFIER($p) WHERE n < 5) - |SELECT * FROM t1""".stripMargin + |SELECT * FROM t1 LIMIT ALL""".stripMargin } checkAnswer(spark.sql(query(":cte"), args = Map("cte" -> "t1")), From 9200bdd971de80160ef8242c21e0585d18bdb785 Mon Sep 17 00:00:00 2001 From: pavle-martinovic_data Date: Mon, 1 Sep 2025 15:13:56 +0200 Subject: [PATCH 06/10] remove old tests, add new, fix --- .../sql/catalyst/optimizer/InlineCTE.scala | 3 + .../optimizer/LimitPushdownSuite.scala | 23 ------ .../analyzer-results/cte-recursion.sql.out | 23 ++++++ .../sql-tests/inputs/cte-recursion.sql | 7 ++ .../sql-tests/results/cte-recursion.sql.out | 72 +++++++++++++++++++ 5 files changed, 105 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala index 91f0b15bbd944..7aabd85ef01dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala @@ -199,6 +199,9 @@ case class InlineCTE( case p @ Project(_, ul: UnionLoop) if ul.id == ref.cteId && ref.isUnlimitedRecursion.isDefined => p.copy(child = ul.copy(limit = Some(-1))) + case ul: UnionLoop + if ul.id == ref.cteId && ref.isUnlimitedRecursion.isDefined => + ul.copy(limit = Some(-1)) case body => body } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala index 2dec2908b18c6..8f6bcc992eda3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala @@ -357,29 +357,6 @@ class LimitPushdownSuite extends PlanTest { } } - test("LimitAll node gets eliminated without UnionLoop") { - val originalQuery = x.join(y.limitAll(), RightOuter).limitAll() - val optimized = Optimize.execute(originalQuery.analyze) - val correctAnswer = x.join(y, RightOuter).analyze - comparePlans(optimized, correctAnswer) - } - - test("LimitAll node gets pushed into UnionLoop") { - val originalQuery = UnionLoop(0, x, y, Seq.empty).limitAll() - val optimized = Optimize.execute(originalQuery.analyze) - val correctAnswer = UnionLoop(0, x, y, Seq.empty, Some(-1)).analyze - comparePlans(optimized, correctAnswer) - } - - test("LimitAll node gets pushed into UnionLoop through other nodes") { - val originalQuery = Union(x, UnionLoop(0, x, y, Seq(ExprId(1), ExprId(2), ExprId(3)) - )).limitAll() - val optimized = Optimize.execute(originalQuery.analyze) - val correctAnswer = Union(x, UnionLoop(0, x, y, Seq(ExprId(1), ExprId(2), ExprId(3)), - Some(-1))).analyze - comparePlans(optimized, correctAnswer) - } - test("LimitAll node gets eliminated before UnionLoop due to previous Limit") { val originalQuery = Union(x, UnionLoop(0, x, y, Seq(ExprId(1), ExprId(2), ExprId(3)) ).localLimit(5)).limitAll() diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out index 9b87d3ae542da..29e6a423fa58d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out @@ -318,6 +318,29 @@ WithCTE +- CTERelationRef xxxx, true, [n#x], false, false, true +-- !query +WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( + SELECT 1 AS n + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) +SELECT * FROM t LIMIT ALL +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, 100 +: +- SubqueryAlias t +: +- UnionLoop xxxx, 100 +: :- Project [1 AS n#x] +: : +- OneRowRelation +: +- Project [(n#x + 1) AS (n + 1)#x] +: +- Filter (n#x < 60) +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [n#x], false ++- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false, true + + -- !query WITH RECURSIVE r(level) AS ( VALUES 0 diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql index 2454653988e91..3bf490a846c99 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql @@ -105,6 +105,13 @@ WITH RECURSIVE t(n) MAX RECURSION LEVEL 100 AS ( ) SELECT * FROM t LIMIT ALL; +WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( + SELECT 1 AS n + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) +SELECT * FROM t LIMIT ALL; + -- terminate recursion with LIMIT WITH RECURSIVE r(level) AS ( VALUES 0 diff --git a/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out index 0563b2d026e09..30e0a42f94a59 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out @@ -405,6 +405,78 @@ struct 9 +-- !query +WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( + SELECT 1 AS n + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) +SELECT * FROM t LIMIT ALL +-- !query schema +struct +-- !query output +1 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +2 +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +3 +30 +31 +32 +33 +34 +35 +36 +37 +38 +39 +4 +40 +41 +42 +43 +44 +45 +46 +47 +48 +49 +5 +50 +51 +52 +53 +54 +55 +56 +57 +58 +59 +6 +60 +7 +8 +9 + + -- !query WITH RECURSIVE r(level) AS ( VALUES 0 From 645dc3bdc0f9b19ed0ae0d1986060ee86857b16c Mon Sep 17 00:00:00 2001 From: pavle-martinovic_data Date: Wed, 3 Sep 2025 15:11:38 +0200 Subject: [PATCH 07/10] make changes according to Wenchen's comments --- .../sql/catalyst/analysis/Analyzer.scala | 12 +- .../sql/catalyst/parser/AstBuilder.scala | 12 +- .../plans/logical/basicLogicalOperators.scala | 8 +- .../analyzer-results/cte-recursion.sql.out | 54 +++++++ .../sql-tests/inputs/cte-recursion.sql | 16 ++ .../sql-tests/results/cte-recursion.sql.out | 152 ++++++++++++++++++ .../apache/spark/sql/ParametersSuite.scala | 2 +- 7 files changed, 242 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5c290824544ef..10e6027975a35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -4289,15 +4289,19 @@ object ApplyLimitAll extends Rule[LogicalPlan] { applyLimitAllToPlan(la.child, isInLimitAll = true) case cteRef: CTERelationRef if isInLimitAll => cteRef.copy(isUnlimitedRecursion = Some(true)) - case other => - other.withNewChildren(other.children + // Allow-list for pushing down Limit All. + case _: Project | _: Filter | _: Join | _: Union | _: Offset | + _: BatchEvalPython | _: ArrowEvalPython | _: SubqueryAlias => + plan.withNewChildren(plan.children .map(child => applyLimitAllToPlan(child, isInLimitAll))) + case other => + other.withNewChildren(plan.children + .map(child => applyLimitAllToPlan(child, isInLimitAll = false))) } } def apply(plan: LogicalPlan): LogicalPlan = { - val ret = applyLimitAllToPlan(plan) - ret + applyLimitAllToPlan(plan) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 957c415f8e91a..1724e5ef7d277 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1346,20 +1346,16 @@ class AstBuilder extends DataTypeAstBuilder } // LIMIT + if (forPipeOperators && clause.nonEmpty && clause != PipeOperators.offsetClause) { + throw QueryParsingErrors.multipleQueryResultClausesWithPipeOperatorsUnsupportedError( + ctx, clause, PipeOperators.limitClause) + } // LIMIT ALL creates LimitAll node which can be used for infinite recursions in recursive CTEs. if (ctx.ALL() != null) { - if (forPipeOperators && clause.nonEmpty && clause != PipeOperators.offsetClause) { - throw QueryParsingErrors.multipleQueryResultClausesWithPipeOperatorsUnsupportedError( - ctx, clause, PipeOperators.limitClause) - } clause = PipeOperators.limitClause LimitAll(withOffset) } else { withOffset.optional(limit) { - if (forPipeOperators && clause.nonEmpty && clause != PipeOperators.offsetClause) { - throw QueryParsingErrors.multipleQueryResultClausesWithPipeOperatorsUnsupportedError( - ctx, clause, PipeOperators.limitClause) - } clause = PipeOperators.limitClause Limit(typedVisit(limit), withOffset) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 8d51efea07a24..28ad71dd03dc5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -1659,7 +1659,13 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends OrderPr copy(child = newChild) } -case class LimitAll(child: LogicalPlan) extends OrderPreservingUnaryNode { +/** Logical node that represents the LIMIT ALL operation. This operation is usually no-op and exists + * to provide compatability with other databases. However, in case of recursive CTEs, Limit nodes + * serve another purpose, to override the default row limit which is determined by a flag. As a + * result, LIMIT ALL should also be used to completely negate the row limit, which is exactly what + * this node is used for. + */ +case class LimitAll(child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output final override val nodePatterns: Seq[TreePattern] = Seq(LIMIT) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out index 29e6a423fa58d..3dced088ca1fc 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out @@ -341,6 +341,60 @@ WithCTE +- CTERelationRef xxxx, true, [n#x], false, false, true +-- !query +WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( + SELECT 1 AS n + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) + (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, 100 +: +- SubqueryAlias t +: +- UnionLoop xxxx, 100 +: :- Project [1 AS n#x] +: : +- OneRowRelation +: +- Project [(n#x + 1) AS (n + 1)#x] +: +- Filter (n#x < 60) +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [n#x], false ++- Union false, false + :- Project [n#x] + : +- SubqueryAlias t + : +- CTERelationRef xxxx, true, [n#x], false, false, true + +- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( + SELECT 1 AS n + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) + (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t LIMIT ALL) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, 100 +: +- SubqueryAlias t +: +- UnionLoop xxxx, 100 +: :- Project [1 AS n#x] +: : +- OneRowRelation +: +- Project [(n#x + 1) AS (n + 1)#x] +: +- Filter (n#x < 60) +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [n#x], false ++- Union false, false + :- Project [n#x] + : +- SubqueryAlias t + : +- CTERelationRef xxxx, true, [n#x], false, false, true + +- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false, true + + -- !query WITH RECURSIVE r(level) AS ( VALUES 0 diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql index 3bf490a846c99..4aa7be79a058d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql @@ -112,6 +112,22 @@ WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( ) SELECT * FROM t LIMIT ALL; +-- One reference is limit all but other isn't. Should fail. +WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( + SELECT 1 AS n + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) + (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t); + +-- One references are limit all. +WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( + SELECT 1 AS n + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) + (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t LIMIT ALL); + -- terminate recursion with LIMIT WITH RECURSIVE r(level) AS ( VALUES 0 diff --git a/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out index 30e0a42f94a59..536fc6c4ea635 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out @@ -477,6 +477,158 @@ struct 9 +-- !query +WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( + SELECT 1 AS n + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) + (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "RECURSION_ROW_LIMIT_EXCEEDED", + "sqlState" : "42836", + "messageParameters" : { + "rowLimit" : "50" + } +} + + +-- !query +WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( + SELECT 1 AS n + UNION ALL + SELECT n + 1 FROM t WHERE n < 60 + ) + (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t LIMIT ALL) +-- !query schema +struct +-- !query output +1 +1 +10 +10 +11 +11 +12 +12 +13 +13 +14 +14 +15 +15 +16 +16 +17 +17 +18 +18 +19 +19 +2 +2 +20 +20 +21 +21 +22 +22 +23 +23 +24 +24 +25 +25 +26 +26 +27 +27 +28 +28 +29 +29 +3 +3 +30 +30 +31 +31 +32 +32 +33 +33 +34 +34 +35 +35 +36 +36 +37 +37 +38 +38 +39 +39 +4 +4 +40 +40 +41 +41 +42 +42 +43 +43 +44 +44 +45 +45 +46 +46 +47 +47 +48 +48 +49 +49 +5 +5 +50 +50 +51 +51 +52 +52 +53 +53 +54 +54 +55 +55 +56 +56 +57 +57 +58 +58 +59 +59 +6 +6 +60 +60 +7 +7 +8 +8 +9 +9 + + -- !query WITH RECURSIVE r(level) AS ( VALUES 0 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index 22b1599cc35aa..0c2380a2d90e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -792,7 +792,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession { | SELECT 1 | UNION ALL | SELECT n+1 FROM IDENTIFIER($p) WHERE n < 5) - |SELECT * FROM t1 LIMIT ALL""".stripMargin + |SELECT * FROM t1""".stripMargin } checkAnswer(spark.sql(query(":cte"), args = Map("cte" -> "t1")), From 11ba72d27e13964667700b4a59d3258ebe46e9df Mon Sep 17 00:00:00 2001 From: pavle-martinovic_data Date: Wed, 10 Sep 2025 20:04:20 +0200 Subject: [PATCH 08/10] remove optional and make changes according to wenchen's comments; regen golden files --- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../sql/catalyst/optimizer/InlineCTE.scala | 34 +++-- .../sql/catalyst/parser/AstBuilder.scala | 3 +- .../plans/logical/basicLogicalOperators.scala | 11 +- .../catalyst/plans/logical/cteOperators.scala | 2 +- .../optimizer/LimitPushdownSuite.scala | 11 +- .../spark/sql/execution/UnionLoopExec.scala | 2 +- .../collations-aliases.sql.out | 4 +- .../analyzer-results/cte-command.sql.out | 10 +- .../analyzer-results/cte-nested.sql.out | 64 ++++---- .../analyzer-results/cte-nonlegacy.sql.out | 38 ++--- .../analyzer-results/cte-recursion.sql.out | 140 +++++++++--------- .../sql-tests/analyzer-results/cte.sql.out | 90 +++++------ .../double-quoted-identifiers-enabled.sql.out | 4 +- .../analyzer-results/group-by-alias.sql.out | 4 +- .../analyzer-results/group-by-all.sql.out | 4 +- .../analyzer-results/group-by-ordinal.sql.out | 4 +- .../identifier-clause.sql.out | 6 +- .../analyzer-results/join-lateral.sql.out | 4 +- .../listagg-collations.sql.out | 10 +- .../analyzer-results/listagg.sql.out | 24 +-- .../non-excludable-rule.sql.out | 2 +- .../postgreSQL/window_part3.sql.out | 8 +- .../analyzer-results/postgreSQL/with.sql.out | 110 +++++++------- .../sql-session-variables.sql.out | 2 +- .../analyzer-results/sql-udf.sql.out | 4 +- .../exists-subquery/exists-cte.sql.out | 16 +- .../in-subquery/in-multiple-columns.sql.out | 4 +- .../subquery/in-subquery/in-with-cte.sql.out | 64 ++++---- .../scalar-subquery-select.sql.out | 20 +-- .../analyzer-results/transform.sql.out | 4 +- .../analyzer-results/using-join.sql.out | 4 +- .../apache/spark/sql/ParametersSuite.scala | 12 +- 33 files changed, 360 insertions(+), 363 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 10e6027975a35..0f3d537e23a0d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -4288,10 +4288,10 @@ object ApplyLimitAll extends Rule[LogicalPlan] { case la: LimitAll => applyLimitAllToPlan(la.child, isInLimitAll = true) case cteRef: CTERelationRef if isInLimitAll => - cteRef.copy(isUnlimitedRecursion = Some(true)) + cteRef.copy(isUnlimitedRecursion = true) // Allow-list for pushing down Limit All. case _: Project | _: Filter | _: Join | _: Union | _: Offset | - _: BatchEvalPython | _: ArrowEvalPython | _: SubqueryAlias => + _: BaseEvalPython | _: Aggregate | _: Window | _: SubqueryAlias => plan.withNewChildren(plan.children .map(child => applyLimitAllToPlan(child, isInLimitAll))) case other => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala index 7aabd85ef01dc..68d4dc45709b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.analysis.DeduplicateRelations import org.apache.spark.sql.catalyst.expressions.{Alias, OuterReference, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, Join, JoinHint, LogicalPlan, Project, Subquery, SubqueryAlias, UnionLoop, WithCTE} +import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, Join, JoinHint, LogicalPlan, Project, Subquery, UnionLoop, WithCTE} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{CTE, PLAN_EXPRESSION} @@ -189,21 +189,11 @@ case class InlineCTE( case ref: CTERelationRef => val refInfo = cteMap(ref.cteId) - val cteBody = refInfo.cteDef.child match { - case sa @ SubqueryAlias(_, ul: UnionLoop) - if ul.id == ref.cteId && ref.isUnlimitedRecursion.isDefined => - sa.copy(child = ul.copy(limit = Some(-1))) - case sa @ SubqueryAlias(_, p @ Project(_, ul: UnionLoop)) - if ul.id == ref.cteId && ref.isUnlimitedRecursion.isDefined => - sa.copy(child = p.copy(child = ul.copy(limit = Some(-1)))) - case p @ Project(_, ul: UnionLoop) - if ul.id == ref.cteId && ref.isUnlimitedRecursion.isDefined => - p.copy(child = ul.copy(limit = Some(-1))) - case ul: UnionLoop - if ul.id == ref.cteId && ref.isUnlimitedRecursion.isDefined => - ul.copy(limit = Some(-1)) - case body => - body + + val cteBody = if (ref.isUnlimitedRecursion) { + setUnlimitedRecursion(refInfo.cteDef.child, ref.cteId) + } else { + refInfo.cteDef.child } if (refInfo.shouldInline) { if (ref.outputSet == refInfo.cteDef.outputSet) { @@ -242,6 +232,18 @@ case class InlineCTE( case _ => plan } } + + // Helper function to set unlimited recursion. + private def setUnlimitedRecursion(plan: LogicalPlan, id: Long): LogicalPlan = { + plan match { + case ul: UnionLoop if ul.id == id => + // Since there is exactly one UnionLoop node with this id in the CTE body, we can stop the + // recursion here. + ul.copy(limit = Some(-1)) + case other => + other.withNewChildren(plan.children.map(child => setUnlimitedRecursion(child, id))) + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 1724e5ef7d277..4887c16c52617 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1346,7 +1346,8 @@ class AstBuilder extends DataTypeAstBuilder } // LIMIT - if (forPipeOperators && clause.nonEmpty && clause != PipeOperators.offsetClause) { + if (forPipeOperators && clause.nonEmpty + && clause != PipeOperators.offsetClause && limit != null) { throw QueryParsingErrors.multipleQueryResultClausesWithPipeOperatorsUnsupportedError( ctx, clause, PipeOperators.limitClause) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 28ad71dd03dc5..7fc717ec2b09a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -1659,11 +1659,12 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends OrderPr copy(child = newChild) } -/** Logical node that represents the LIMIT ALL operation. This operation is usually no-op and exists - * to provide compatability with other databases. However, in case of recursive CTEs, Limit nodes - * serve another purpose, to override the default row limit which is determined by a flag. As a - * result, LIMIT ALL should also be used to completely negate the row limit, which is exactly what - * this node is used for. +/** + * Logical node that represents the LIMIT ALL operation. This operation is usually no-op and exists + * to provide compatability with other databases. However, in case of recursive CTEs, Limit nodes + * serve another purpose, to override the default row limit which is determined by a flag. As a + * result, LIMIT ALL should also be used to completely negate the row limit, which is exactly what + * this node is used for. */ case class LimitAll(child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala index 59dab7b8f4db8..373f8b3b98d20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala @@ -195,7 +195,7 @@ case class CTERelationRef( statsOpt: Option[Statistics] = None, recursive: Boolean = false, override val maxRows: Option[Long] = None, - isUnlimitedRecursion: Option[Boolean] = None) extends LeafNode with MultiInstanceRelation { + isUnlimitedRecursion: Boolean = false) extends LeafNode with MultiInstanceRelation { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala index 8f6bcc992eda3..2dcab5cfd29c4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Add, ExprId, GenericInternalRow} +import org.apache.spark.sql.catalyst.expressions.{Add, GenericInternalRow} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -356,13 +356,4 @@ class LimitPushdownSuite extends PlanTest { comparePlans(Optimize.execute(originalQuery2), originalQuery2) } } - - test("LimitAll node gets eliminated before UnionLoop due to previous Limit") { - val originalQuery = Union(x, UnionLoop(0, x, y, Seq(ExprId(1), ExprId(2), ExprId(3)) - ).localLimit(5)).limitAll() - val optimized = Optimize.execute(originalQuery.analyze) - val correctAnswer = Union(x, UnionLoop(0, x, y, Seq(ExprId(1), ExprId(2), ExprId(3)), - Some(5))).analyze - comparePlans(optimized, correctAnswer) - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala index c714917eed4ed..c6d4091fc8092 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala @@ -106,7 +106,7 @@ case class UnionLoopExec( private def executeAndCacheAndCount(plan: LogicalPlan, currentLimit: Int) = { // In case limit is defined, we create a (local) limit node above the plan and execute // the newly created plan. - val planWithLimit = if (limit.isDefined && limit.get != -1) { + val planWithLimit = if (limit.isDefined && limit.get >= 0) { LocalLimit(Literal(currentLimit), plan) } else { plan diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out index 454682e392be8..b10cf5298440d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out @@ -183,7 +183,7 @@ WithCTE : +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet +- Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false + +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false, false -- !query @@ -232,7 +232,7 @@ Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- LocalLimit 1 : +- Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x] : +- SubqueryAlias cte -: +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false +: +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false, false +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out index 5ca0f8ae253af..bb0fef48abf7d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out @@ -10,7 +10,7 @@ CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`cte_tbl`, ErrorI : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -32,7 +32,7 @@ CreateViewCommand `cte_view`, WITH s AS (SELECT 42 AS col) SELECT * FROM s, fals : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -49,7 +49,7 @@ Project [col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -64,7 +64,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d : +- OneRowRelation +- Project [col#x] +- SubqueryAlias S - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -86,7 +86,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out index 3c48bd406eedf..43bc4bb77fddd 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out @@ -15,10 +15,10 @@ WithCTE : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query @@ -37,7 +37,7 @@ Aggregate [max(c#x) AS max(c)#x] : +- OneRowRelation +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false, false, 1 + +- CTERelationRef xxxx, true, [c#x], false, false, 1, false -- !query @@ -54,7 +54,7 @@ Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- OneRowRelation @@ -140,10 +140,10 @@ WithCTE : +- SubqueryAlias t2 : +- Project [2#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [2#x], false, false, 1 +: +- CTERelationRef xxxx, true, [2#x], false, false, 1, false +- Project [2#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [2#x], false, false, 1 + +- CTERelationRef xxxx, true, [2#x], false, false, 1, false -- !query @@ -178,11 +178,11 @@ WithCTE : : : +- OneRowRelation : : +- Project [c#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c#x], false, false, 1 +: : +- CTERelationRef xxxx, true, [c#x], false, false, 1, false : +- OneRowRelation +- Project [scalarsubquery()#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false, 1 + +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false, 1, false -- !query @@ -215,15 +215,15 @@ WithCTE : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [3#x], false, false, 1 +: +- CTERelationRef xxxx, true, [3#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [3#x], false, false, 1 +: +- CTERelationRef xxxx, true, [3#x], false, false, 1, false +- Project [3#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [3#x], false, false, 1 + +- CTERelationRef xxxx, true, [3#x], false, false, 1, false -- !query @@ -248,7 +248,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false, false, 1 + +- CTERelationRef xxxx, true, [c#x], false, false, 1, false -- !query @@ -277,7 +277,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false, false, 1 + +- CTERelationRef xxxx, true, [c#x], false, false, 1, false -- !query @@ -312,7 +312,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false, false, 1 + +- CTERelationRef xxxx, true, [c#x], false, false, 1, false -- !query @@ -335,7 +335,7 @@ WithCTE : : +- OneRowRelation : +- Project [2#x] : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [2#x], false, false, 1 + : +- CTERelationRef xxxx, true, [2#x], false, false, 1, false +- OneRowRelation @@ -362,7 +362,7 @@ WithCTE : : : +- OneRowRelation : : +- Project [2#x] : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [2#x], false, false, 1 + : : +- CTERelationRef xxxx, true, [2#x], false, false, 1, false : +- OneRowRelation +- OneRowRelation @@ -396,7 +396,7 @@ WithCTE : : : +- OneRowRelation : : +- Project [3#x] : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [3#x], false, false, 1 + : : +- CTERelationRef xxxx, true, [3#x], false, false, 1, false : +- OneRowRelation +- OneRowRelation @@ -425,9 +425,9 @@ WithCTE : : +- OneRowRelation : +- Project [c#x] : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [c#x], false, false, 1 + : +- CTERelationRef xxxx, true, [c#x], false, false, 1, false +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false, false, 1 + +- CTERelationRef xxxx, true, [c#x], false, false, 1, false -- !query @@ -448,14 +448,14 @@ WithCTE : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query @@ -480,10 +480,10 @@ WithCTE : +- SubqueryAlias t : +- Project [2#x] : +- SubqueryAlias aBC -: +- CTERelationRef xxxx, true, [2#x], false, false, 1 +: +- CTERelationRef xxxx, true, [2#x], false, false, 1, false +- Project [2#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [2#x], false, false, 1 + +- CTERelationRef xxxx, true, [2#x], false, false, 1, false -- !query @@ -506,7 +506,7 @@ WithCTE : : +- OneRowRelation : +- Project [2#x] : +- SubqueryAlias aBC - : +- CTERelationRef xxxx, true, [2#x], false, false, 1 + : +- CTERelationRef xxxx, true, [2#x], false, false, 1, false +- OneRowRelation @@ -530,15 +530,15 @@ WithCTE : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- Project [1#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query @@ -561,12 +561,12 @@ WithCTE : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query @@ -594,19 +594,19 @@ WithCTE : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias __auto_generated_subquery_name : +- Project [1#x] : +- SubqueryAlias cte_inner_inner -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out index 70d00a1916bef..0e831ef820ebe 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out @@ -15,10 +15,10 @@ WithCTE : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query @@ -37,7 +37,7 @@ Aggregate [max(c#x) AS max(c)#x] : +- OneRowRelation +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false, false, 1 + +- CTERelationRef xxxx, true, [c#x], false, false, 1, false -- !query @@ -54,7 +54,7 @@ Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- OneRowRelation @@ -171,11 +171,11 @@ WithCTE : : : +- OneRowRelation : : +- Project [c#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c#x], false, false, 1 +: : +- CTERelationRef xxxx, true, [c#x], false, false, 1, false : +- OneRowRelation +- Project [scalarsubquery()#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false, 1 + +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false, 1, false -- !query @@ -225,7 +225,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false, false, 1 + +- CTERelationRef xxxx, true, [c#x], false, false, 1, false -- !query @@ -254,7 +254,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false, false, 1 + +- CTERelationRef xxxx, true, [c#x], false, false, 1, false -- !query @@ -289,7 +289,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false, false, 1 + +- CTERelationRef xxxx, true, [c#x], false, false, 1, false -- !query @@ -392,14 +392,14 @@ WithCTE : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query @@ -462,15 +462,15 @@ WithCTE : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- Project [1#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query @@ -493,12 +493,12 @@ WithCTE : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query @@ -526,19 +526,19 @@ WithCTE : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias __auto_generated_subquery_name : +- Project [1#x] : +- SubqueryAlias cte_inner_inner -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out index 3dced088ca1fc..848cdce88b0ba 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out @@ -44,7 +44,7 @@ WithCTE : +- UnionLoopRef xxxx, [level#x], false +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -68,7 +68,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x], false +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -93,7 +93,7 @@ WithCTE : +- UnionLoopRef xxxx, [a#x], false +- Project [c#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [c#x], false, false + +- CTERelationRef xxxx, true, [c#x], false, false, false -- !query @@ -116,7 +116,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x], false +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -160,7 +160,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [n#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -190,7 +190,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x], false +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -238,7 +238,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x], false +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -284,7 +284,7 @@ WithCTE +- LocalLimit 60 +- Project [n#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -365,7 +365,7 @@ WithCTE : +- CTERelationRef xxxx, true, [n#x], false, false, true +- Project [n#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -417,7 +417,7 @@ WithCTE +- LocalLimit 10 +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -487,7 +487,7 @@ WithCTE +- Sort [level#x ASC NULLS FIRST], true +- Project [level#x, level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -537,7 +537,7 @@ WithCTE : +- OneRowRelation +- Project [n#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false, 2 + +- CTERelationRef xxxx, true, [n#x], false, false, 2, false -- !query @@ -616,7 +616,7 @@ WithCTE +- LocalLimit 5 +- Project [n#x] +- SubqueryAlias t1 - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -645,10 +645,10 @@ WithCTE : : +- UnionLoopRef xxxx, [level#x], false : +- Project [(level + 1)#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [(level + 1)#x], false, false +: +- CTERelationRef xxxx, true, [(level + 1)#x], false, false, false +- Project [level#x] +- SubqueryAlias t1 - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -692,15 +692,15 @@ WithCTE : : +- UnionLoopRef xxxx, [level#x], false : +- Project [(level + 1)#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [(level + 1)#x], false, false +: +- CTERelationRef xxxx, true, [(level + 1)#x], false, false, false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [level#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [level#x], false, false +: +- CTERelationRef xxxx, true, [level#x], false, false, false +- Project [level#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -739,7 +739,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x, col2#x], false +- Project [level#x, data#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x, data#x], false, false + +- CTERelationRef xxxx, true, [level#x, data#x], false, false, false -- !query @@ -772,7 +772,7 @@ WithCTE : +- OneRowRelation +- Project [level#x, data#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x, data#x], false, false + +- CTERelationRef xxxx, true, [level#x, data#x], false, false, false -- !query @@ -825,7 +825,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x, col2#x], false +- Project [level#x, data#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x, data#x], false, false + +- CTERelationRef xxxx, true, [level#x, data#x], false, false, false -- !query @@ -898,7 +898,7 @@ WithCTE : +- OneRowRelation +- Project [level#x, data#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x, data#x], false, false + +- CTERelationRef xxxx, true, [level#x, data#x], false, false, false -- !query @@ -950,7 +950,7 @@ WithCTE : +- OneRowRelation +- Project [level#x, data#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x, data#x], false, false + +- CTERelationRef xxxx, true, [level#x, data#x], false, false, false -- !query @@ -1029,7 +1029,7 @@ SELECT * FROM r, false, false, LocalTempView, UNSUPPORTED, true : +- UnionLoopRef xxxx, [col1#x], false +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -1052,7 +1052,7 @@ Project [level#x] : +- UnionLoopRef xxxx, [col1#x], false +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -1089,7 +1089,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d : +- UnionLoopRef xxxx, [col1#x], false +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -1138,11 +1138,11 @@ WithCTE :- InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/rt2, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/rt2], Append, `spark_catalog`.`default`.`rt2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/rt2), [level] : +- Project [level#x] : +- SubqueryAlias r - : +- CTERelationRef xxxx, true, [level#x], false, false + : +- CTERelationRef xxxx, true, [level#x], false, false, false +- InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/rt2, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/rt2], Append, `spark_catalog`.`default`.`rt2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/rt2), [level] +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -1198,9 +1198,9 @@ WithCTE +- Project [level#x, level#x] +- Join Inner, (level#x = (level#x + 10)) :- SubqueryAlias r1 - : +- CTERelationRef xxxx, true, [level#x], false, false + : +- CTERelationRef xxxx, true, [level#x], false, false, false +- SubqueryAlias r2 - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -1227,10 +1227,10 @@ WithCTE +- Join Inner, (level#x = level#x) :- SubqueryAlias r1 : +- SubqueryAlias r - : +- CTERelationRef xxxx, true, [level#x], false, false + : +- CTERelationRef xxxx, true, [level#x], false, false, false +- SubqueryAlias r2 +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -1262,14 +1262,14 @@ WithCTE : +- UnionLoop xxxx : :- Project [0 AS outerlevel#x, innerlevel#x] : : +- SubqueryAlias r1 -: : +- CTERelationRef xxxx, true, [innerlevel#x], false, false +: : +- CTERelationRef xxxx, true, [innerlevel#x], false, false, false : +- Project [(outerlevel#x + 1) AS (outerlevel + 1)#x, innerlevel#x] : +- Filter (outerlevel#x < 3) : +- SubqueryAlias r2 : +- UnionLoopRef xxxx, [outerlevel#x, innerlevel#x], false +- Project [outerlevel#x, innerlevel#x] +- SubqueryAlias r2 - +- CTERelationRef xxxx, true, [outerlevel#x, innerlevel#x], false, false + +- CTERelationRef xxxx, true, [outerlevel#x, innerlevel#x], false, false, false -- !query @@ -1303,14 +1303,14 @@ WithCTE : +- Union false, false : :- Project [level#x] : : +- SubqueryAlias r -: : +- CTERelationRef xxxx, true, [level#x], false, false +: : +- CTERelationRef xxxx, true, [level#x], false, false, false : +- Project [(level#x + 1) AS (level + 1)#x] : +- Filter (level#x < 3) : +- SubqueryAlias r -: +- CTERelationRef xxxx, true, [level#x], false, false +: +- CTERelationRef xxxx, true, [level#x], false, false, false +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -1344,7 +1344,7 @@ WithCTE : : : +- UnionLoopRef xxxx, [col1#x], false : : +- Project [level#x] : : +- SubqueryAlias r -: : +- CTERelationRef xxxx, true, [level#x], false, false +: : +- CTERelationRef xxxx, true, [level#x], false, false, false : +- Project [(level#x + 1) AS (level + 1)#x] : +- Filter (level#x < 3) : +- SubqueryAlias r @@ -1352,7 +1352,7 @@ WithCTE : +- UnionLoopRef xxxx, [level#x], false +- Project [level#x] +- SubqueryAlias r - +- CTERelationRef xxxx, true, [level#x], false, false + +- CTERelationRef xxxx, true, [level#x], false, false, false -- !query @@ -1400,7 +1400,7 @@ WithCTE : +- UnionLoopRef xxxx, [destination#x, path#x, length#x], false +- Project [destination#x, path#x, length#x] +- SubqueryAlias destinations_from_new_york - +- CTERelationRef xxxx, true, [destination#x, path#x, length#x], false, false + +- CTERelationRef xxxx, true, [destination#x, path#x, length#x], false, false, false -- !query @@ -1430,7 +1430,7 @@ WithCTE +- Sort [a#x ASC NULLS FIRST], true +- Project [a#x] +- SubqueryAlias fibonacci - +- CTERelationRef xxxx, true, [a#x, b#x], false, false + +- CTERelationRef xxxx, true, [a#x, b#x], false, false, false -- !query @@ -1455,7 +1455,7 @@ WithCTE +- LocalLimit 5 +- Project [a#x] +- SubqueryAlias t1 - +- CTERelationRef xxxx, true, [a#x, b#x, c#x], false, false + +- CTERelationRef xxxx, true, [a#x, b#x, c#x], false, false, false -- !query @@ -1498,7 +1498,7 @@ WithCTE +- LocalLimit 63 +- Project [n#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -1541,10 +1541,10 @@ WithCTE : : +- Project [id#x AS id#x, xid#x AS xid#x] : : +- UnionLoopRef xxxx, [id#x, xid#x], false : +- SubqueryAlias x -: +- CTERelationRef xxxx, true, [id#x], false, false, 2 +: +- CTERelationRef xxxx, true, [id#x], false, false, 2, false +- Project [id#x, xid#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [id#x, xid#x], false, false + +- CTERelationRef xxxx, true, [id#x, xid#x], false, false, false -- !query @@ -1585,10 +1585,10 @@ WithCTE : : +- Project [1#x AS n#x] : : +- UnionLoopRef xxxx, [1#x], false : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [a#x, b#x], false, false +: +- CTERelationRef xxxx, true, [a#x, b#x], false, false, false +- Project [n#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -1626,14 +1626,14 @@ WithCTE : : +- Project [j#x AS k#x] : : +- Project [j#x] : : +- SubqueryAlias t2 -: : +- CTERelationRef xxxx, true, [j#x], false, false +: : +- CTERelationRef xxxx, true, [j#x], false, false, false : +- Project [k#x] : +- Filter (k#x <= 5) : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [k#x], false, false +: +- CTERelationRef xxxx, true, [k#x], false, false, false +- Project [n#x] +- SubqueryAlias t1 - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -1666,7 +1666,7 @@ WithCTE : +- UnionLoop xxxx : :- Project [0 AS outerlevel#x, innerlevel#x] : : +- SubqueryAlias r1 -: : +- CTERelationRef xxxx, true, [innerlevel#x], false, false +: : +- CTERelationRef xxxx, true, [innerlevel#x], false, false, false : +- Project [(outerlevel1#x + 1) AS (outerlevel1 + 1)#x, innerlevel1#x] : +- Filter (outerlevel1#x < 3) : +- SubqueryAlias r2 @@ -1674,7 +1674,7 @@ WithCTE : +- UnionLoopRef xxxx, [outerlevel#x, innerlevel#x], false +- Project [outerlevel1#x, innerlevel1#x] +- SubqueryAlias r2 - +- CTERelationRef xxxx, true, [outerlevel1#x, innerlevel1#x], false, false + +- CTERelationRef xxxx, true, [outerlevel1#x, innerlevel1#x], false, false, false -- !query @@ -1729,7 +1729,7 @@ WithCTE : :- Project [x#x, x#x] : : +- Filter (x#x = 1) : : +- SubqueryAlias tmp -: : +- CTERelationRef xxxx, true, [x#x], false, false, 5 +: : +- CTERelationRef xxxx, true, [x#x], false, false, 5, false : +- Project [(x#x + 1) AS (x + 1)#x, x#x] : +- Filter (x#x < 5) : +- SubqueryAlias rcte @@ -1737,7 +1737,7 @@ WithCTE : +- UnionLoopRef xxxx, [x#x, x#x], false +- Project [x#x, y#x] +- SubqueryAlias rcte - +- CTERelationRef xxxx, true, [x#x, y#x], false, false + +- CTERelationRef xxxx, true, [x#x, y#x], false, false, false -- !query @@ -1768,7 +1768,7 @@ WithCTE : :- Project [x#x, x#x] : : +- Filter (x#x = 1) : : +- SubqueryAlias tmp -: : +- CTERelationRef xxxx, true, [x#x], false, false, 5 +: : +- CTERelationRef xxxx, true, [x#x], false, false, 5, false : +- Project [(x#x + 1) AS (x + 1)#x, x#x] : +- Filter (x#x < 5) : +- SubqueryAlias rcte @@ -1776,7 +1776,7 @@ WithCTE : +- UnionLoopRef xxxx, [x#x, x#x], false +- Project [x#x, y#x] +- SubqueryAlias rcte - +- CTERelationRef xxxx, true, [x#x, y#x], false, false + +- CTERelationRef xxxx, true, [x#x, y#x], false, false, false -- !query @@ -1807,7 +1807,7 @@ WithCTE : :- Project [x#x, x#x, x#x, x#x] : : +- Filter (x#x = 1) : : +- SubqueryAlias tmp -: : +- CTERelationRef xxxx, true, [x#x], false, false, 5 +: : +- CTERelationRef xxxx, true, [x#x], false, false, 5, false : +- Project [(x#x + 1) AS (x + 1)#x, x#x, (y#x + 1) AS (y + 1)#x, y#x] : +- Filter (x#x < 5) : +- SubqueryAlias rcte @@ -1815,7 +1815,7 @@ WithCTE : +- UnionLoopRef xxxx, [x#x, x#x, x#x, x#x], false +- Project [x#x, y#x, z#x, t#x] +- SubqueryAlias rcte - +- CTERelationRef xxxx, true, [x#x, y#x, z#x, t#x], false, false + +- CTERelationRef xxxx, true, [x#x, y#x, z#x, t#x], false, false, false -- !query @@ -1912,7 +1912,7 @@ WithCTE : +- UnionLoopRef xxxx, [1#x, CAST(1 AS BIGINT)#xL], false +- Project [n#x, m#xL] +- SubqueryAlias t1 - +- CTERelationRef xxxx, true, [n#x, m#xL], false, false + +- CTERelationRef xxxx, true, [n#x, m#xL], false, false, false -- !query @@ -1956,7 +1956,7 @@ WithCTE +- LocalLimit 25 +- Project [n#x] +- SubqueryAlias t1 - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -1984,14 +1984,14 @@ WithCTE : : +- SubqueryAlias __auto_generated_subquery_name : : +- Project [n#x] : : +- SubqueryAlias t1 - : : +- CTERelationRef xxxx, true, [n#x], false, false + : : +- CTERelationRef xxxx, true, [n#x], false, false, false : +- Aggregate [sum(n#x) AS sum(n)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- GlobalLimit 3 : +- LocalLimit 3 : +- Project [n#x] : +- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [n#x], false, false + : +- CTERelationRef xxxx, true, [n#x], false, false, false +- OneRowRelation @@ -2021,14 +2021,14 @@ WithCTE : : +- LocalLimit 5 : : +- Project [n#x] : : +- SubqueryAlias t1 - : : +- CTERelationRef xxxx, true, [n#x], false, false + : : +- CTERelationRef xxxx, true, [n#x], false, false, false : +- Aggregate [sum(n#x) AS sum(n)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- GlobalLimit 3 : +- LocalLimit 3 : +- Project [n#x] : +- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [n#x], false, false + : +- CTERelationRef xxxx, true, [n#x], false, false, false +- OneRowRelation @@ -2066,7 +2066,7 @@ WithCTE : : : +- LocalLimit 10 : : : +- Project [n#x] : : : +- SubqueryAlias t1 -: : : +- CTERelationRef xxxx, true, [n#x], false, false +: : : +- CTERelationRef xxxx, true, [n#x], false, false, false : : +- OneRowRelation : +- Project [(m#xL + scalar-subquery#x []) AS (m + scalarsubquery())#xL] : : +- Aggregate [sum(n#x) AS sum(n)#xL] @@ -2075,7 +2075,7 @@ WithCTE : : +- LocalLimit 3 : : +- Project [n#x] : : +- SubqueryAlias t1 -: : +- CTERelationRef xxxx, true, [n#x], false, false +: : +- CTERelationRef xxxx, true, [n#x], false, false, false : +- SubqueryAlias t2 : +- Project [scalarsubquery()#xL AS m#xL] : +- UnionLoopRef xxxx, [scalarsubquery()#xL], false @@ -2083,7 +2083,7 @@ WithCTE +- LocalLimit 20 +- Project [m#xL] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [m#xL], false, false + +- CTERelationRef xxxx, true, [m#xL], false, false, false -- !query @@ -2110,10 +2110,10 @@ WithCTE +- Union false, false :- Project [n#x] : +- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [n#x], false, false + : +- CTERelationRef xxxx, true, [n#x], false, false, false +- Project [n#x] +- SubqueryAlias t1 - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -2232,4 +2232,4 @@ WithCTE : +- UnionLoopRef xxxx, [n#x], false +- Project [n#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out index 8d8d978cb0cbb..3b2cafd15c5bf 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out @@ -86,7 +86,7 @@ WithCTE : +- LocalRelation [id#x] +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false, false + +- CTERelationRef xxxx, true, [1#x], false, false, false -- !query @@ -147,13 +147,13 @@ WithCTE : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [id#x], false, false +: +- CTERelationRef xxxx, true, [id#x], false, false, false +- Project [id#x, 2#x] +- Join Cross :- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [id#x], false, false + : +- CTERelationRef xxxx, true, [id#x], false, false, false +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [2#x], false, false + +- CTERelationRef xxxx, true, [2#x], false, false, false -- !query @@ -191,10 +191,10 @@ WithCTE +- Join Cross :- SubqueryAlias t1 : +- SubqueryAlias CTE1 - : +- CTERelationRef xxxx, true, [id#x], false, false + : +- CTERelationRef xxxx, true, [id#x], false, false, false +- SubqueryAlias t2 +- SubqueryAlias CTE1 - +- CTERelationRef xxxx, true, [id#x], false, false + +- CTERelationRef xxxx, true, [id#x], false, false, false -- !query @@ -210,7 +210,7 @@ WithCTE +- Project [x#x] +- Filter (x#x = 1) +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x], false, false, 1 + +- CTERelationRef xxxx, true, [x#x], false, false, 1, false -- !query @@ -226,7 +226,7 @@ WithCTE +- Project [x#x, y#x] +- Filter ((x#x = 1) AND (y#x = 2)) +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1 + +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1, false -- !query @@ -241,7 +241,7 @@ WithCTE : +- OneRowRelation +- Project [x#x, x#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x, x#x], false, false, 1 + +- CTERelationRef xxxx, true, [x#x, x#x], false, false, 1, false -- !query @@ -344,46 +344,46 @@ WithCTE : +- Project [c8#x AS c7#x] : +- Project [c8#x] : +- SubqueryAlias w8 -: +- CTERelationRef xxxx, true, [c8#x], false, false, 1 +: +- CTERelationRef xxxx, true, [c8#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias w6 : +- Project [c7#x AS c6#x] : +- Project [c7#x] : +- SubqueryAlias w7 -: +- CTERelationRef xxxx, true, [c7#x], false, false, 1 +: +- CTERelationRef xxxx, true, [c7#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias w5 : +- Project [c6#x AS c5#x] : +- Project [c6#x] : +- SubqueryAlias w6 -: +- CTERelationRef xxxx, true, [c6#x], false, false, 1 +: +- CTERelationRef xxxx, true, [c6#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias w4 : +- Project [c5#x AS c4#x] : +- Project [c5#x] : +- SubqueryAlias w5 -: +- CTERelationRef xxxx, true, [c5#x], false, false, 1 +: +- CTERelationRef xxxx, true, [c5#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias w3 : +- Project [c4#x AS c3#x] : +- Project [c4#x] : +- SubqueryAlias w4 -: +- CTERelationRef xxxx, true, [c4#x], false, false, 1 +: +- CTERelationRef xxxx, true, [c4#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias w2 : +- Project [c3#x AS c2#x] : +- Project [c3#x] : +- SubqueryAlias w3 -: +- CTERelationRef xxxx, true, [c3#x], false, false, 1 +: +- CTERelationRef xxxx, true, [c3#x], false, false, 1, false :- CTERelationDef xxxx, false : +- SubqueryAlias w1 : +- Project [c2#x AS c1#x] : +- Project [c2#x] : +- SubqueryAlias w2 -: +- CTERelationRef xxxx, true, [c2#x], false, false, 1 +: +- CTERelationRef xxxx, true, [c2#x], false, false, 1, false +- Project [c1#x] +- SubqueryAlias w1 - +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false -- !query @@ -420,7 +420,7 @@ WithCTE +- Project [42#x, 10#x] +- Join Inner :- SubqueryAlias same_name - : +- CTERelationRef xxxx, true, [42#x], false, false, 1 + : +- CTERelationRef xxxx, true, [42#x], false, false, 1, false +- SubqueryAlias same_name +- Project [10 AS 10#x] +- OneRowRelation @@ -459,7 +459,7 @@ WithCTE : +- OneRowRelation +- Project [x#x, typeof(x#x) AS typeof(x)#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x], false, false, 1 + +- CTERelationRef xxxx, true, [x#x], false, false, 1, false -- !query @@ -519,7 +519,7 @@ Project [y#x] : +- OneRowRelation +- Project [(x#x + 1) AS y#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x], false, false, 1 + +- CTERelationRef xxxx, true, [x#x], false, false, 1, false -- !query @@ -533,7 +533,7 @@ Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- OneRowRelation : +- Project [x#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [x#x], false, false, 1 +: +- CTERelationRef xxxx, true, [x#x], false, false, 1, false +- OneRowRelation @@ -548,7 +548,7 @@ Project [1 IN (list#x []) AS (1 IN (listquery()))#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- OneRowRelation @@ -596,14 +596,14 @@ WithCTE :- Join Inner : :- SubqueryAlias x : : +- SubqueryAlias T1 - : : +- CTERelationRef xxxx, true, [a#x], false, false, 1 + : : +- CTERelationRef xxxx, true, [a#x], false, false, 1, false : +- SubqueryAlias y : +- Project [b#x] : +- SubqueryAlias T1 - : +- CTERelationRef xxxx, true, [b#x], false, false, 1 + : +- CTERelationRef xxxx, true, [b#x], false, false, 1, false +- SubqueryAlias z +- SubqueryAlias T1 - +- CTERelationRef xxxx, true, [a#x], false, false, 1 + +- CTERelationRef xxxx, true, [a#x], false, false, 1, false -- !query @@ -631,9 +631,9 @@ WithCTE +- Project [c#x, a#x] +- Join Inner :- SubqueryAlias ttTT - : +- CTERelationRef xxxx, true, [c#x], false, false, 1 + : +- CTERelationRef xxxx, true, [c#x], false, false, 1, false +- SubqueryAlias tttT_2 - +- CTERelationRef xxxx, true, [a#x], false, false, 1 + +- CTERelationRef xxxx, true, [a#x], false, false, 1, false -- !query @@ -649,7 +649,7 @@ Project [scalar-subquery#x [x#x] AS scalarsubquery(x)#x] : : +- OneRowRelation : +- Project [x#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [x#x], false, false, 1 +: +- CTERelationRef xxxx, true, [x#x], false, false, 1, false +- SubqueryAlias T +- Project [1 AS x#x, 2 AS y#x] +- OneRowRelation @@ -668,7 +668,7 @@ Project [scalar-subquery#x [x#x && y#x] AS scalarsubquery(x, y)#x] : : +- OneRowRelation : +- Project [((outer(x#x) + outer(y#x)) + z#x) AS ((outer(T.x) + outer(T.y)) + z)#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [z#x], false, false, 1 +: +- CTERelationRef xxxx, true, [z#x], false, false, 1, false +- SubqueryAlias T +- Project [1 AS x#x, 2 AS y#x] +- OneRowRelation @@ -688,12 +688,12 @@ WithCTE : +- SubqueryAlias q2 : +- Project [x#x] : +- SubqueryAlias q1 -: +- CTERelationRef xxxx, true, [x#x], false, false, 1 +: +- CTERelationRef xxxx, true, [x#x], false, false, 1, false +- Project [x#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [x#x] +- SubqueryAlias q2 - +- CTERelationRef xxxx, true, [x#x], false, false, 1 + +- CTERelationRef xxxx, true, [x#x], false, false, 1, false -- !query @@ -710,12 +710,12 @@ WithCTE : +- SubqueryAlias q1 : +- Project [(x#x + 1) AS (x + 1)#x] : +- SubqueryAlias q1 -: +- CTERelationRef xxxx, true, [x#x], false, false, 1 +: +- CTERelationRef xxxx, true, [x#x], false, false, 1, false +- Project [(x + 1)#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [(x + 1)#x] +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [(x + 1)#x], false, false, 1 + +- CTERelationRef xxxx, true, [(x + 1)#x], false, false, 1, false -- !query @@ -757,9 +757,9 @@ WithCTE : +- Aggregate [max(j#x) AS max(j)#x] : +- SubqueryAlias cte2 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [j#x], false, false + : +- CTERelationRef xxxx, true, [j#x], false, false, false +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [j#x], false, false + +- CTERelationRef xxxx, true, [j#x], false, false, false -- !query @@ -827,10 +827,10 @@ Project [1#x] +- Union false, false :- Project [1#x] : +- SubqueryAlias cte - : +- CTERelationRef xxxx, true, [1#x], false, false, 1 + : +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- Project [2#x] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [2#x], false, false, 1 + +- CTERelationRef xxxx, true, [2#x], false, false, 1, false -- !query @@ -892,7 +892,7 @@ Union false, false : : : +- OneRowRelation : : +- Project [1#x] : : +- SubqueryAlias cte -: : +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: : +- CTERelationRef xxxx, true, [1#x], false, false, 1, false : +- LocalRelation [col1#x] +- Project [col1#x] +- Filter exists#x [] @@ -903,7 +903,7 @@ Union false, false : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias cte - : +- CTERelationRef xxxx, true, [1#x], false, false, 1 + : +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- LocalRelation [col1#x] @@ -943,7 +943,7 @@ WithCTE : : : : +- OneRowRelation : : : +- Project [1#x] : : : +- SubqueryAlias cte -: : : +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: : : +- CTERelationRef xxxx, true, [1#x], false, false, 1, false : : +- LocalRelation [col1#x] : +- Project [col1#x] : +- Filter exists#x [] @@ -954,11 +954,11 @@ WithCTE : : : +- OneRowRelation : : +- Project [1#x] : : +- SubqueryAlias cte -: : +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: : +- CTERelationRef xxxx, true, [1#x], false, false, 1, false : +- LocalRelation [col1#x] +- Project [col1#x] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [col1#x], false, false, 2 + +- CTERelationRef xxxx, true, [col1#x], false, false, 2, false -- !query @@ -1000,7 +1000,7 @@ Project [col1#x] : : : +- OneRowRelation : : +- Project [1#x] : : +- SubqueryAlias cte - : : +- CTERelationRef xxxx, true, [1#x], false, false, 1 + : : +- CTERelationRef xxxx, true, [1#x], false, false, 1, false : +- LocalRelation [col1#x] +- Project [col1#x] +- Filter exists#x [] @@ -1011,7 +1011,7 @@ Project [col1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias cte - : +- CTERelationRef xxxx, true, [1#x], false, false, 1 + : +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- LocalRelation [col1#x] @@ -1028,7 +1028,7 @@ WithCTE : +- OneRowRelation +- Project [1#x] +- SubqueryAlias `a.b.c` - +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +- CTERelationRef xxxx, true, [1#x], false, false, 1, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out index 51569e6c965c8..12f48c14e0a54 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out @@ -438,7 +438,7 @@ CreateViewCommand `myview`, [(c1,None)], WITH "v"("a") AS (SELECT 1) SELECT "a" : +- OneRowRelation +- Project [a#x] +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x], false, false, 1 + +- CTERelationRef xxxx, true, [a#x], false, false, 1, false -- !query @@ -458,7 +458,7 @@ Project [a1#x AS a2#x] : +- OneRowRelation +- Project [a#x] +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x], false, false, 1 + +- CTERelationRef xxxx, true, [a#x], false, false, 1, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-alias.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-alias.sql.out index a3fe0bedfa247..55d77106dca2b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-alias.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-alias.sql.out @@ -368,7 +368,7 @@ Aggregate [scalar-subquery#x []], [scalar-subquery#x [] AS subq1#x] : : : +- OneRowRelation : : +- Project [1#x] : : +- SubqueryAlias cte -: : +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: : +- CTERelationRef xxxx, true, [1#x], false, false, 1, false : +- WithCTE : :- CTERelationDef xxxx, false : : +- SubqueryAlias cte @@ -376,7 +376,7 @@ Aggregate [scalar-subquery#x []], [scalar-subquery#x [] AS subq1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias cte -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- LocalRelation [col1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out index 7837ba426d951..b6b8fdfe0d3fe 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out @@ -416,7 +416,7 @@ Aggregate [scalar-subquery#x []], [scalar-subquery#x [] AS subq1#x] : : : +- OneRowRelation : : +- Project [1#x] : : +- SubqueryAlias cte -: : +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: : +- CTERelationRef xxxx, true, [1#x], false, false, 1, false : +- WithCTE : :- CTERelationDef xxxx, false : : +- SubqueryAlias cte @@ -424,7 +424,7 @@ Aggregate [scalar-subquery#x []], [scalar-subquery#x [] AS subq1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias cte -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- LocalRelation [col1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out index 430e645040c2d..72d8f64141aaa 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out @@ -489,7 +489,7 @@ Aggregate [scalar-subquery#x []], [scalar-subquery#x [] AS subq1#x] : : : +- OneRowRelation : : +- Project [1#x] : : +- SubqueryAlias cte -: : +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: : +- CTERelationRef xxxx, true, [1#x], false, false, 1, false : +- WithCTE : :- CTERelationDef xxxx, false : : +- SubqueryAlias cte @@ -497,7 +497,7 @@ Aggregate [scalar-subquery#x []], [scalar-subquery#x [] AS subq1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias cte -: +- CTERelationRef xxxx, true, [1#x], false, false, 1 +: +- CTERelationRef xxxx, true, [1#x], false, false, 1, false +- LocalRelation [col1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index 38b28f2d3b232..8f0676f7733a4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -1022,7 +1022,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Aggregate [max(c1#x) AS max(c1)#x] +- SubqueryAlias T - +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2 + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2, false -- !query @@ -1041,7 +1041,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Aggregate [max(c1#x) AS max(c1)#x] +- SubqueryAlias T - +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2 + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2, false -- !query @@ -1055,7 +1055,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Aggregate [max(c1#x) AS max(c1)#x] +- SubqueryAlias ABC - +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2 + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index 390fcf5e30152..e5acfcb88fc71 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -1377,10 +1377,10 @@ WithCTE : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : : +- LocalRelation [col1#x, col2#x] : +- SubqueryAlias cte1 -: +- CTERelationRef xxxx, true, [c1#x], false, false +: +- CTERelationRef xxxx, true, [c1#x], false, false, false +- Project [c1#x, c2#x] +- SubqueryAlias cte2 - +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out index 7a7d7aa3f0472..6fe453a5d5eb9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out @@ -20,7 +20,7 @@ WithCTE : +- LocalRelation [col1#x] +- Project [len(c1#x) AS len(c1)#x, regexp_count(c1#x, a) AS regexp_count(c1, a)#x, regexp_count(c1#x, b) AS regexp_count(c1, b)#x, regexp_count(c1#x, A) AS regexp_count(c1, A)#x, regexp_count(c1#x, B) AS regexp_count(c1, B)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false -- !query @@ -37,7 +37,7 @@ WithCTE , $) AS replace(replace(c1, , ), , $)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false -- !query @@ -61,7 +61,7 @@ WithCTE : +- LocalRelation [col1#x] +- Project [len(c1#x) AS len(c1)#x, regexp_count(c1#x, a) AS regexp_count(c1, a)#x, regexp_count(c1#x, b) AS regexp_count(c1, b)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false -- !query @@ -85,7 +85,7 @@ WithCTE : +- LocalRelation [col1#x] +- Project [len(c1#x) AS len(c1)#x, regexp_count(c1#x, a) AS regexp_count(c1, a)#x, regexp_count(c1#x, xbc) AS regexp_count(c1, xbc)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false -- !query @@ -102,7 +102,7 @@ WithCTE , $) AS replace(replace(c1, , ), , $)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out index 1d78d882cf6a8..e22fc1ff24d85 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out @@ -35,7 +35,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x, regexp_count(col#x, c) AS regexp_count(col, c)#x, regexp_count(col#x, d) AS regexp_count(col, d)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false + +- CTERelationRef xxxx, true, [col#x], false, false, false -- !query @@ -54,7 +54,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x, regexp_count(col#x, c) AS regexp_count(col, c)#x, regexp_count(col#x, d) AS regexp_count(col, d)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false + +- CTERelationRef xxxx, true, [col#x], false, false, false -- !query @@ -73,7 +73,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x, regexp_count(col#x, c) AS regexp_count(col, c)#x, regexp_count(col#x, d) AS regexp_count(col, d)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false + +- CTERelationRef xxxx, true, [col#x], false, false, false -- !query @@ -105,7 +105,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x, regexp_count(col#x, c) AS regexp_count(col, c)#x, regexp_count(col#x, d) AS regexp_count(col, d)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false + +- CTERelationRef xxxx, true, [col#x], false, false, false -- !query @@ -124,7 +124,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -143,7 +143,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -213,7 +213,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [concat(cast((col#x = baba) as string), cast((col#x = bbaa) as string)) AS concat((col = baba), (col = bbaa))#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -232,7 +232,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [concat(cast((col#x = b|a|b|a) as string), cast((col#x = b|b|a|a) as string)) AS concat((col = b|a|b|a), (col = b|b|a|a))#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -283,7 +283,7 @@ WithCTE : +- LocalRelation [col1#x] +- Project [len(col#x) AS len(col)#x, regexp_count(cast(col#x as string), cast(0xDEAD as string)) AS regexp_count(col, X'DEAD')#x, regexp_count(cast(col#x as string), cast(0xBEEF as string)) AS regexp_count(col, X'BEEF')#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -298,7 +298,7 @@ WithCTE : +- LocalRelation [col1#x] +- Project [len(col#x) AS len(col)#x, regexp_count(cast(col#x as string), cast(0xDEAD as string)) AS regexp_count(col, X'DEAD')#x, regexp_count(cast(col#x as string), cast(0xBEEF as string)) AS regexp_count(col, X'BEEF')#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -313,7 +313,7 @@ WithCTE : +- LocalRelation [col1#x] +- Project [len(col#x) AS len(col)#x, regexp_count(cast(col#x as string), cast(0x42 as string)) AS regexp_count(col, X'42')#x, regexp_count(cast(col#x as string), cast(0xDEAD as string)) AS regexp_count(col, X'DEAD')#x, regexp_count(cast(col#x as string), cast(0xBEEF as string)) AS regexp_count(col, X'BEEF')#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col#x], false, false, 1 + +- CTERelationRef xxxx, true, [col#x], false, false, 1, false -- !query @@ -332,7 +332,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [len(col1#x) AS len(col1)#x, regexp_count(col1#x, 1) AS regexp_count(col1, 1)#x, regexp_count(col1#x, 2) AS regexp_count(col1, 2)#x, regexp_count(col1#x, 3) AS regexp_count(col1, 3)#x, len(col2#x) AS len(col2)#x, regexp_count(col2#x, true) AS regexp_count(col2, true)#x, regexp_count(col1#x, false) AS regexp_count(col1, false)#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [col1#x, col2#x], false, false, 1 + +- CTERelationRef xxxx, true, [col1#x, col2#x], false, false, 1, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out index 6bfbe53ace4f5..6ae281bfac78f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out @@ -47,7 +47,7 @@ WithCTE +- Filter (id#xL > scalar-subquery#x []) : +- Aggregate [max(id#xL) AS max(id)#xL] : +- SubqueryAlias tmp - : +- CTERelationRef xxxx, true, [id#xL], false, false, 2 + : +- CTERelationRef xxxx, true, [id#xL], false, false, 2, false +- Range (0, 3, step=1) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out index bc6dc828ad857..4df9bef4723b5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out @@ -99,7 +99,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false, false, 18 + +- CTERelationRef xxxx, true, [x#xL], false, false, 18, false -- !query @@ -121,7 +121,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false, false, 18 + +- CTERelationRef xxxx, true, [x#xL], false, false, 18, false -- !query @@ -154,7 +154,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false, false, 26 + +- CTERelationRef xxxx, true, [x#xL], false, false, 26, false -- !query @@ -187,7 +187,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false, false, 26 + +- CTERelationRef xxxx, true, [x#xL], false, false, 26, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out index edbdcba851483..e56f8ace4ca23 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out @@ -12,10 +12,10 @@ WithCTE +- Project [x#x, y#x, x#x, y#x] +- Join Inner :- SubqueryAlias q1 - : +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1 + : +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1, false +- SubqueryAlias q2 +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1 + +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1, false -- !query @@ -50,7 +50,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x], false +- Aggregate [sum(n#x) AS sum(n)#xL] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -76,7 +76,7 @@ WithCTE : +- UnionLoopRef xxxx, [scalarsubquery()#x], false +- Project [n#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -107,7 +107,7 @@ SELECT * FROM nums, false, false, LocalTempView, UNSUPPORTED, true : +- UnionLoopRef xxxx, [col1#x], false +- Project [n#x] +- SubqueryAlias nums - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -130,7 +130,7 @@ Project [n#x] : +- UnionLoopRef xxxx, [col1#x], false +- Project [n#x] +- SubqueryAlias nums - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -161,7 +161,7 @@ SELECT * FROM nums, false, true, LocalTempView, UNSUPPORTED, true : +- UnionLoopRef xxxx, [col1#x], false +- Project [n#x] +- SubqueryAlias nums - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -184,7 +184,7 @@ Project [n#x] : +- UnionLoopRef xxxx, [col1#x], false +- Project [n#x] +- SubqueryAlias nums - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -229,7 +229,7 @@ WithCTE +- LocalLimit 10 +- Project [n#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -264,7 +264,7 @@ WithCTE : +- OneRowRelation +- Project [x#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x], false, false, 1 + +- CTERelationRef xxxx, true, [x#x], false, false, 1, false -- !query @@ -289,7 +289,7 @@ WithCTE : +- UnionLoopRef xxxx, [foo#x], false +- Project [n#x AS is_text#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -417,7 +417,7 @@ WithCTE +- Sort [name#x ASC NULLS FIRST], true +- Project [root_name#x, id#x, parent_department#x, name#x] +- SubqueryAlias subdepartment - +- CTERelationRef xxxx, true, [root_name#x, id#x, parent_department#x, name#x], false, false + +- CTERelationRef xxxx, true, [root_name#x, id#x, parent_department#x, name#x], false, false, false -- !query @@ -454,7 +454,7 @@ WithCTE +- Sort [name#x ASC NULLS FIRST], true +- Project [level#x, id#x, parent_department#x, name#x] +- SubqueryAlias subdepartment - +- CTERelationRef xxxx, true, [level#x, id#x, parent_department#x, name#x], false, false + +- CTERelationRef xxxx, true, [level#x, id#x, parent_department#x, name#x], false, false, false -- !query @@ -492,7 +492,7 @@ WithCTE +- Project [level#x, id#x, parent_department#x, name#x] +- Filter (level#x >= 2) +- SubqueryAlias subdepartment - +- CTERelationRef xxxx, true, [level#x, id#x, parent_department#x, name#x], false, false + +- CTERelationRef xxxx, true, [level#x, id#x, parent_department#x, name#x], false, false, false -- !query @@ -512,7 +512,7 @@ WithCTE +- Sort [name#x ASC NULLS FIRST], true +- Project [id#x, parent_department#x, name#x] +- SubqueryAlias subdepartment - +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false + +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false, false -- !query @@ -554,7 +554,7 @@ Aggregate [count(1) AS count(1)#xL] : +- Project [n#x] : +- Filter (n#x < 50000) : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [n#x], false, false + : +- CTERelationRef xxxx, true, [n#x], false, false, false +- SubqueryAlias t +- WithCTE :- CTERelationDef xxxx, false @@ -570,7 +570,7 @@ Aggregate [count(1) AS count(1)#xL] : +- UnionLoopRef xxxx, [1#x], false +- Project [n#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -597,9 +597,9 @@ WithCTE : +- Aggregate [(cast(sum(y#xL) as double) / cast(100 as double)) AS (sum(y) / 100)#x] : +- SubqueryAlias qsub : +- SubqueryAlias q1 - : +- CTERelationRef xxxx, true, [x#x, y#xL], false, false + : +- CTERelationRef xxxx, true, [x#x, y#xL], false, false, false +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [x#x, y#xL], false, false + +- CTERelationRef xxxx, true, [x#x, y#xL], false, false, false -- !query @@ -640,7 +640,7 @@ CreateViewCommand `vsubdepartment`, WITH RECURSIVE subdepartment AS : +- UnionLoopRef xxxx, [id#x, parent_department#x, name#x], false +- Project [id#x, parent_department#x, name#x] +- SubqueryAlias subdepartment - +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false + +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false, false -- !query @@ -670,7 +670,7 @@ Sort [name#x ASC NULLS FIRST], true : +- UnionLoopRef xxxx, [id#x, parent_department#x, name#x], false +- Project [id#x, parent_department#x, name#x] +- SubqueryAlias subdepartment - +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false + +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false, false -- !query @@ -701,7 +701,7 @@ SELECT sum(n) AS sum FROM t, false, false, PersistedView, COMPENSATION, true : +- UnionLoopRef xxxx, [col1#x], false +- Aggregate [sum(n#x) AS sum#xL] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -724,7 +724,7 @@ Project [sum#xL] : +- UnionLoopRef xxxx, [col1#x], false +- Aggregate [sum(n#x) AS sum#xL] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -756,7 +756,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x, col2#x], false +- Project [i#x, j#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [i#x, j#x], false, false + +- CTERelationRef xxxx, true, [i#x, j#x], false, false, false -- !query @@ -809,10 +809,10 @@ WithCTE +- Join Inner, (((path#x[0] = path#x[0]) AND (size(path#x, false) = 1)) AND (size(path#x, false) > 1)) :- SubqueryAlias t1 : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [id#x, path#x], false, false + : +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false +- SubqueryAlias t2 +- SubqueryAlias t - +- CTERelationRef xxxx, true, [id#x, path#x], false, false + +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false -- !query @@ -847,10 +847,10 @@ WithCTE +- Join Inner, (((path#x[0] = path#x[0]) AND (size(path#x, false) = 1)) AND (size(path#x, false) > 1)) :- SubqueryAlias t1 : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [id#x, path#x], false, false + : +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false +- SubqueryAlias t2 +- SubqueryAlias t - +- CTERelationRef xxxx, true, [id#x, path#x], false, false + +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false -- !query @@ -880,10 +880,10 @@ WithCTE +- Join Inner, (id#x = id#x) :- SubqueryAlias t1 : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [id#x, path#x], false, false + : +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false +- SubqueryAlias t2 +- SubqueryAlias t - +- CTERelationRef xxxx, true, [id#x, path#x], false, false + +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false -- !query @@ -937,7 +937,7 @@ WithCTE : +- UnionLoopRef xxxx, [f#x, t#x, label#x, array(struct(f, t))#x, false#x], false +- Project [f#x, t#x, label#x, path#x, cycle#x] +- SubqueryAlias search_graph - +- CTERelationRef xxxx, true, [f#x, t#x, label#x, path#x, cycle#x], false, false + +- CTERelationRef xxxx, true, [f#x, t#x, label#x, path#x, cycle#x], false, false, false -- !query @@ -972,7 +972,7 @@ WithCTE +- Sort [path#x ASC NULLS FIRST], true +- Project [f#x, t#x, label#x, path#x, cycle#x] +- SubqueryAlias search_graph - +- CTERelationRef xxxx, true, [f#x, t#x, label#x, path#x, cycle#x], false, false + +- CTERelationRef xxxx, true, [f#x, t#x, label#x, path#x, cycle#x], false, false, false -- !query @@ -992,7 +992,7 @@ WithCTE : +- UnionLoop xxxx : :- Project [id#x] : : +- SubqueryAlias y -: : +- CTERelationRef xxxx, true, [id#x], false, false, 1 +: : +- CTERelationRef xxxx, true, [id#x], false, false, 1, false : +- Project [(id#x + 1) AS (id + 1)#x] : +- Filter (id#x < 5) : +- SubqueryAlias x @@ -1000,7 +1000,7 @@ WithCTE : +- UnionLoopRef xxxx, [id#x], false +- Project [id#x] +- SubqueryAlias x - +- CTERelationRef xxxx, true, [id#x], false, false + +- CTERelationRef xxxx, true, [id#x], false, false, false -- !query @@ -1031,17 +1031,17 @@ WithCTE : +- Union false, false : :- Project [id#x] : : +- SubqueryAlias x -: : +- CTERelationRef xxxx, true, [id#x], false, false +: : +- CTERelationRef xxxx, true, [id#x], false, false, false : +- Project [id#x] : +- SubqueryAlias x -: +- CTERelationRef xxxx, true, [id#x], false, false +: +- CTERelationRef xxxx, true, [id#x], false, false, false :- CTERelationDef xxxx, false : +- SubqueryAlias z : +- Project [id#x AS id#x] : +- UnionLoop xxxx : :- Project [id#x] : : +- SubqueryAlias x -: : +- CTERelationRef xxxx, true, [id#x], false, false +: : +- CTERelationRef xxxx, true, [id#x], false, false, false : +- Project [(id#x + 1) AS (id + 1)#x] : +- Filter (id#x < 10) : +- SubqueryAlias z @@ -1049,7 +1049,7 @@ WithCTE : +- UnionLoopRef xxxx, [id#x], false +- Project [id#x] +- SubqueryAlias z - +- CTERelationRef xxxx, true, [id#x], false, false + +- CTERelationRef xxxx, true, [id#x], false, false, false -- !query @@ -1080,17 +1080,17 @@ WithCTE : +- Union false, false : :- Project [id#x] : : +- SubqueryAlias x -: : +- CTERelationRef xxxx, true, [id#x], false, false +: : +- CTERelationRef xxxx, true, [id#x], false, false, false : +- Project [id#x] : +- SubqueryAlias x -: +- CTERelationRef xxxx, true, [id#x], false, false +: +- CTERelationRef xxxx, true, [id#x], false, false, false :- CTERelationDef xxxx, false : +- SubqueryAlias z : +- Project [id#x AS id#x] : +- UnionLoop xxxx : :- Project [id#x] : : +- SubqueryAlias y -: : +- CTERelationRef xxxx, true, [id#x], false, false +: : +- CTERelationRef xxxx, true, [id#x], false, false, false : +- Project [(id#x + 1) AS (id + 1)#x] : +- Filter (id#x < 10) : +- SubqueryAlias z @@ -1098,7 +1098,7 @@ WithCTE : +- UnionLoopRef xxxx, [id#x], false +- Project [id#x] +- SubqueryAlias z - +- CTERelationRef xxxx, true, [id#x], false, false + +- CTERelationRef xxxx, true, [id#x], false, false, false -- !query @@ -1268,7 +1268,7 @@ WithCTE : +- UnionLoopRef xxxx, [1#x], false +- Project [n#x] +- SubqueryAlias x - +- CTERelationRef xxxx, true, [n#x], false, false + +- CTERelationRef xxxx, true, [n#x], false, false, false -- !query @@ -1338,7 +1338,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x], false +- Project [id#x] +- SubqueryAlias x - +- CTERelationRef xxxx, true, [id#x], false, false + +- CTERelationRef xxxx, true, [id#x], false, false, false -- !query @@ -1392,7 +1392,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x], false +- Project [i#x] +- SubqueryAlias foo - +- CTERelationRef xxxx, true, [i#x], false, false + +- CTERelationRef xxxx, true, [i#x], false, false, false -- !query @@ -1426,7 +1426,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x], false +- Project [i#x] +- SubqueryAlias foo - +- CTERelationRef xxxx, true, [i#x], false, false + +- CTERelationRef xxxx, true, [i#x], false, false, false -- !query @@ -1457,7 +1457,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x], false +- Project [i#x] +- SubqueryAlias foo - +- CTERelationRef xxxx, true, [i#x], false, false + +- CTERelationRef xxxx, true, [i#x], false, false, false -- !query @@ -1488,7 +1488,7 @@ WithCTE : +- UnionLoopRef xxxx, [col1#x], false +- Project [i#x] +- SubqueryAlias foo - +- CTERelationRef xxxx, true, [i#x], false, false + +- CTERelationRef xxxx, true, [i#x], false, false, false -- !query @@ -1522,7 +1522,7 @@ WithCTE +- SubqueryAlias q +- Project [foo#x] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [foo#x], false, false, 1 + +- CTERelationRef xxxx, true, [foo#x], false, false, 1, false -- !query @@ -1556,7 +1556,7 @@ WithCTE : +- UnionLoop xxxx : :- Project [i#x] : : +- SubqueryAlias s -: : +- CTERelationRef xxxx, true, [i#x], false, false +: : +- CTERelationRef xxxx, true, [i#x], false, false, false : +- Project [(j#x + 1) AS (j + 1)#x] : +- Filter (j#x < 10) : +- SubqueryAlias t @@ -1564,7 +1564,7 @@ WithCTE : +- UnionLoopRef xxxx, [i#x], false +- Project [j#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [j#x], false, false + +- CTERelationRef xxxx, true, [j#x], false, false, false -- !query @@ -1592,13 +1592,13 @@ WithCTE : +- Union false, false : :- Project [2#x] : : +- SubqueryAlias innermost -: : +- CTERelationRef xxxx, true, [2#x], false, false, 1 +: : +- CTERelationRef xxxx, true, [2#x], false, false, 1, false : +- Project [3 AS 3#x] : +- OneRowRelation +- Sort [x#x ASC NULLS FIRST], true +- Project [x#x] +- SubqueryAlias outermost - +- CTERelationRef xxxx, true, [x#x], false, false, 3 + +- CTERelationRef xxxx, true, [x#x], false, false, 3, false -- !query @@ -1826,7 +1826,7 @@ WithCTE : +- OneRowRelation +- Project [x#x] +- SubqueryAlias ordinality - +- CTERelationRef xxxx, true, [x#x], false, false, 1 + +- CTERelationRef xxxx, true, [x#x], false, false, 1, false -- !query @@ -1867,7 +1867,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d : +- OneRowRelation +- Project [42#x] +- SubqueryAlias test - +- CTERelationRef xxxx, true, [42#x], false, false, 1 + +- CTERelationRef xxxx, true, [42#x], false, false, 1, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out index ae538ce0bb778..455ffc0ad6be7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out @@ -2118,7 +2118,7 @@ WithCTE : +- OneRowRelation +- Project [c1#x AS 1#x] +- SubqueryAlias v1 - +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out index 2b8a47c9ca633..4fd3b58afdd15 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out @@ -2803,7 +2803,7 @@ WithCTE : +- LocalRelation [col1#x, col2#x] +- Project [spark_catalog.default.foo3_1a(c1, c2)#x] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [spark_catalog.default.foo3_1a(c1, c2)#x], false, false + +- CTERelationRef xxxx, true, [spark_catalog.default.foo3_1a(c1, c2)#x], false, false, false -- !query @@ -3528,7 +3528,7 @@ Project [spark_catalog.default.foo3_2e1(occurrences#x, instance_start_time#x) AS : : +- OneRowRelation : +- Project [CASE WHEN (isnull(outer(occurrences#x)) OR (size(outer(occurrences#x), false) = 0)) THEN cast(null as string) ELSE sort_array(diffs#x, true)[0].id END AS id#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [diffs#x], false, false, 1 +: +- CTERelationRef xxxx, true, [diffs#x], false, false, 1, false +- Project [cast(array(struct(col1, 2022-01-01 10:11:12, col2, 1), struct(col1, 2022-01-01 10:11:15, col2, 2)) as array>) AS occurrences#x, cast(2022-01-01 as timestamp) AS instance_start_time#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out index abaf6a2432251..aaa098d334006 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out @@ -133,7 +133,7 @@ WithCTE : +- Filter (outer(emp_name#x) = emp_name#x) : +- SubqueryAlias b : +- SubqueryAlias bonus_cte - : +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], false, false + : +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], false, false, false +- SubqueryAlias a +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) @@ -189,10 +189,10 @@ WithCTE : +- Join Inner, (dept_id#x = dept_id#x) : :- SubqueryAlias a : : +- SubqueryAlias emp_cte - : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false, false : +- SubqueryAlias b : +- SubqueryAlias dept_cte - : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false, false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] @@ -253,10 +253,10 @@ WithCTE : +- Join LeftOuter, (dept_id#x = dept_id#x) : :- SubqueryAlias a : : +- SubqueryAlias emp_cte - : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false, false : +- SubqueryAlias b : +- SubqueryAlias dept_cte - : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false, false +- Join Inner :- Join Inner : :- SubqueryAlias b @@ -268,7 +268,7 @@ WithCTE : : +- LocalRelation [emp_name#x, bonus_amt#x] : +- SubqueryAlias e : +- SubqueryAlias emp_cte - : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false + : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false, false +- SubqueryAlias d +- SubqueryAlias dept +- View (`DEPT`, [dept_id#x, dept_name#x, state#x]) @@ -322,7 +322,7 @@ WithCTE : +- Filter (count(1)#xL > cast(1 as bigint)) : +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] : +- SubqueryAlias empdept - : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false, false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] @@ -375,7 +375,7 @@ WithCTE : +- Filter (count(1)#xL < cast(1 as bigint)) : +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] : +- SubqueryAlias empdept - : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false, false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out index 230ffc005e90d..5b50d64fc0531 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out @@ -330,7 +330,7 @@ WithCTE +- Project [t1a#x, t1b#x, t1a#x, t1b#x] +- Join Inner, (t1b#x = t1b#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out index 199b876fb9a86..9b547f45d0895 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out @@ -138,7 +138,7 @@ WithCTE : +- Project [t1b#x] : +- Filter (cast(t1b#x as int) > 0) : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -197,21 +197,21 @@ WithCTE : : : :- Project [t1b#x] : : : : +- Filter (cast(t1b#x as int) > 0) : : : : +- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false : : : +- Project [t1b#x] : : : +- Filter (cast(t1b#x as int) > 5) : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false : : +- Intersect false : : :- Project [t1b#x] : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false : : +- Project [t1b#x] : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false : +- Project [t1b#x] : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -268,22 +268,22 @@ WithCTE : : : :- Join FullOuter, (t1c#x = t1c#x) : : : : :- Join Inner, (t1b#x > t1b#x) : : : : : :- SubqueryAlias cte1 - : : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false + : : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false : : : : : +- SubqueryAlias cte2 : : : : : +- SubqueryAlias cte1 - : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false + : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false : : : : +- SubqueryAlias cte3 : : : : +- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false : : : +- SubqueryAlias cte4 : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false : : +- SubqueryAlias cte5 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false : +- SubqueryAlias cte6 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -354,16 +354,16 @@ WithCTE :- Join FullOuter, (t1a#x = t1a#x) : :- Join Inner, ((cast(t1b#x as int) > 5) AND (t1a#x = t1a#x)) : : :- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false : : +- SubqueryAlias cte2 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false : +- SubqueryAlias cte3 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false +- SubqueryAlias cte4 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false -- !query @@ -424,10 +424,10 @@ WithCTE +- Project [t1a#x, t1b#x] +- Join Inner, (t1h#x >= t1h#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false, false -- !query @@ -485,16 +485,16 @@ WithCTE :- Join RightOuter, (t1b#x = t1b#x) : :- Join Inner, (t1a#x = t1a#x) : : :- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false, false : : +- SubqueryAlias cte2 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false, false : +- SubqueryAlias cte3 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false, false +- SubqueryAlias cte4 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false, false -- !query @@ -538,10 +538,10 @@ WithCTE +- Project [t1a#x, t1b#x] +- Join RightOuter, (t1a#x = t1a#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false -- !query @@ -599,15 +599,15 @@ WithCTE : : +- SubqueryAlias t1 : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false +- SubqueryAlias s +- Project [t1b#x] +- Join LeftOuter, (t1b#x = t1b#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false -- !query @@ -642,7 +642,7 @@ WithCTE : +- Project [t1b#x] : +- Filter (cast(t1b#x as int) < 0) : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -722,16 +722,16 @@ WithCTE : :- Join RightOuter, (t1b#x = t1b#x) : : :- Join Inner, (t1a#x = t1a#x) : : : :- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false, false : : : +- SubqueryAlias cte2 : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false, false : : +- SubqueryAlias cte3 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false, false : +- SubqueryAlias cte4 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out index f64b3736b5521..9d6a817e0dc2d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -623,7 +623,7 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x] : : +- OneRowRelation : +- Project [(a#x + outer(c1#x)) AS (a + outer(t1.c1))#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [a#x], false, false, 1 +: +- CTERelationRef xxxx, true, [a#x], false, false, 1, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -647,7 +647,7 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : : +- LocalRelation [c1#x, c2#x] : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -677,10 +677,10 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : : +- Project [c1#x, c2#x] : : +- Filter (outer(c1#x) = c1#x) : : +- SubqueryAlias t3 -: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : +- SubqueryAlias t4 -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -713,10 +713,10 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- Union false, false : :- Project [c1#x, c2#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false : +- Project [c2#x, c1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -756,9 +756,9 @@ WithCTE : : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : : +- Filter (c1#x = outer(c1#x)) : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false + : : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false : +- SubqueryAlias v - : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false + : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -779,7 +779,7 @@ WithCTE : +- Project [a#x] : +- Filter (a#x = outer(c1#x)) : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [a#x], false, false, 1 + : +- CTERelationRef xxxx, true, [a#x], false, false, 1, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -1027,7 +1027,7 @@ WithCTE : +- Aggregate [sum(1) AS sum(1)#xL] : +- Filter ((a#x = cast(outer(col#x) as int)) OR (upper(cast(outer(col#x) as string)) = Y)) : +- SubqueryAlias T - : +- CTERelationRef xxxx, true, [a#x], false, false, 1 + : +- CTERelationRef xxxx, true, [a#x], false, false, 1, false +- SubqueryAlias foo +- Project [null AS col#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out index d5376913f9ae4..681b537e8e5f2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out @@ -888,10 +888,10 @@ WithCTE +- Join Inner, (b#x = b#x) :- SubqueryAlias t1 : +- SubqueryAlias temp - : +- CTERelationRef xxxx, true, [b#x], false, false + : +- CTERelationRef xxxx, true, [b#x], false, false, false +- SubqueryAlias t2 +- SubqueryAlias temp - +- CTERelationRef xxxx, true, [b#x], false, false + +- CTERelationRef xxxx, true, [b#x], false, false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out index 3b04886afb59e..51bf54e8a3e03 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out @@ -833,6 +833,6 @@ WithCTE +- Project [coalesce(key#x, key#x) AS key#x, key#x] +- Join FullOuter, (key#x = key#x) :- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [key#x], false, false, 1 + : +- CTERelationRef xxxx, true, [key#x], false, false, 1, false +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [key#x], false, false, 1 + +- CTERelationRef xxxx, true, [key#x], false, false, 1, false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index 0c2380a2d90e3..04fa8adb34d4d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -786,13 +786,15 @@ class ParametersSuite extends QueryTest with SharedSparkSession { } test("SPARK-50892: parameterized identifier inside a recursive CTE") { + spark.conf.set("spark.sql.cteRecursionRowLimit", "50") def query(p: String): String = { s""" - |WITH RECURSIVE t1(n) AS ( - | SELECT 1 - | UNION ALL - | SELECT n+1 FROM IDENTIFIER($p) WHERE n < 5) - |SELECT * FROM t1""".stripMargin + |WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( + | SELECT 1 AS n + | UNION ALL + | SELECT n + 1 FROM t WHERE n < 60 + | ) + | SELECT * FROM t LIMIT ALL""".stripMargin } checkAnswer(spark.sql(query(":cte"), args = Map("cte" -> "t1")), From d458986b0a0b9444e963651e69b688d5b3442ffc Mon Sep 17 00:00:00 2001 From: pavle-martinovic_data Date: Thu, 11 Sep 2025 09:12:57 +0200 Subject: [PATCH 09/10] undo test change --- .../scala/org/apache/spark/sql/ParametersSuite.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index 04fa8adb34d4d..5c5b054b98313 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -789,12 +789,11 @@ class ParametersSuite extends QueryTest with SharedSparkSession { spark.conf.set("spark.sql.cteRecursionRowLimit", "50") def query(p: String): String = { s""" - |WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( - | SELECT 1 AS n - | UNION ALL - | SELECT n + 1 FROM t WHERE n < 60 - | ) - | SELECT * FROM t LIMIT ALL""".stripMargin + |WITH RECURSIVE t1(n) AS ( + | SELECT 1 + | UNION ALL + | SELECT n+1 FROM IDENTIFIER($p) WHERE n < 5) + |SELECT * FROM t1""".stripMargin } checkAnswer(spark.sql(query(":cte"), args = Map("cte" -> "t1")), From 96a0e89b1716ef433104186797aaeeeda4dfd4ab Mon Sep 17 00:00:00 2001 From: pavle-martinovic_data Date: Fri, 12 Sep 2025 16:57:15 +0200 Subject: [PATCH 10/10] cleanup --- .../sql/catalyst/analysis/Analyzer.scala | 23 ---------- .../sql/catalyst/analysis/ApplyLimitAll.scala | 44 +++++++++++++++++++ .../sql/catalyst/optimizer/InlineCTE.scala | 2 +- .../apache/spark/sql/ParametersSuite.scala | 2 +- 4 files changed, 46 insertions(+), 25 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyLimitAll.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 0f3d537e23a0d..4678f1c2d2fa8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -4282,29 +4282,6 @@ object RemoveTempResolvedColumn extends Rule[LogicalPlan] { } } -object ApplyLimitAll extends Rule[LogicalPlan] { - def applyLimitAllToPlan(plan: LogicalPlan, isInLimitAll: Boolean = false): LogicalPlan = { - plan match { - case la: LimitAll => - applyLimitAllToPlan(la.child, isInLimitAll = true) - case cteRef: CTERelationRef if isInLimitAll => - cteRef.copy(isUnlimitedRecursion = true) - // Allow-list for pushing down Limit All. - case _: Project | _: Filter | _: Join | _: Union | _: Offset | - _: BaseEvalPython | _: Aggregate | _: Window | _: SubqueryAlias => - plan.withNewChildren(plan.children - .map(child => applyLimitAllToPlan(child, isInLimitAll))) - case other => - other.withNewChildren(plan.children - .map(child => applyLimitAllToPlan(child, isInLimitAll = false))) - } - } - - def apply(plan: LogicalPlan): LogicalPlan = { - applyLimitAllToPlan(plan) - } -} - /** * Rule that's used to handle `UnresolvedHaving` nodes with resolved `condition` and `child`. * It's placed outside the main batch to avoid conflicts with other rules that resolve diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyLimitAll.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyLimitAll.scala new file mode 100644 index 0000000000000..51500c956b6d1 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyLimitAll.scala @@ -0,0 +1,44 @@ +/* + * 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. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, BaseEvalPython, CTERelationRef, Filter, Join, LimitAll, LogicalPlan, Offset, Project, SubqueryAlias, Union, Window} +import org.apache.spark.sql.catalyst.rules.Rule + +object ApplyLimitAll extends Rule[LogicalPlan] { + private def applyLimitAllToPlan(plan: LogicalPlan, isInLimitAll: Boolean = false): LogicalPlan = { + plan match { + case la: LimitAll => + applyLimitAllToPlan(la.child, isInLimitAll = true) + case cteRef: CTERelationRef if isInLimitAll => + cteRef.copy(isUnlimitedRecursion = true) + // Allow-list for pushing down Limit All. + case _: Project | _: Filter | _: Join | _: Union | _: Offset | + _: BaseEvalPython | _: Aggregate | _: Window | _: SubqueryAlias => + plan.withNewChildren(plan.children + .map(child => applyLimitAllToPlan(child, isInLimitAll))) + case other => + other.withNewChildren(plan.children + .map(child => applyLimitAllToPlan(child, isInLimitAll = false))) + } + } + + def apply(plan: LogicalPlan): LogicalPlan = { + applyLimitAllToPlan(plan) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala index 68d4dc45709b3..44b31d3a05b41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala @@ -191,7 +191,7 @@ case class InlineCTE( val refInfo = cteMap(ref.cteId) val cteBody = if (ref.isUnlimitedRecursion) { - setUnlimitedRecursion(refInfo.cteDef.child, ref.cteId) + setUnlimitedRecursion(refInfo.cteDef.child, ref.cteId) } else { refInfo.cteDef.child } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index 5c5b054b98313..c7f47c9da8d41 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -786,7 +786,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession { } test("SPARK-50892: parameterized identifier inside a recursive CTE") { - spark.conf.set("spark.sql.cteRecursionRowLimit", "50") + def query(p: String): String = { s""" |WITH RECURSIVE t1(n) AS (