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..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 @@ -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, 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/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/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/InlineCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala index ad1a1a99b8257..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 @@ -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, UnionLoop, WithCTE} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{CTE, PLAN_EXPRESSION} @@ -189,14 +189,20 @@ case class InlineCTE( case ref: CTERelationRef => val refInfo = cteMap(ref.cteId) + + val cteBody = if (ref.isUnlimitedRecursion) { + setUnlimitedRecursion(refInfo.cteDef.child, ref.cteId) + } else { + refInfo.cteDef.child + } 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) @@ -226,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/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/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 4e930280381c1..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,15 +1346,23 @@ class AstBuilder extends DataTypeAstBuilder } // LIMIT - // - LIMIT ALL is the same as omitting the LIMIT clause - withOffset.optional(limit) { - if (forPipeOperators && clause.nonEmpty && clause != PipeOperators.offsetClause) { - throw QueryParsingErrors.multipleQueryResultClausesWithPipeOperatorsUnsupportedError( - ctx, clause, PipeOperators.limitClause) - } + if (forPipeOperators && clause.nonEmpty + && clause != PipeOperators.offsetClause && limit != null) { + 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) { clause = PipeOperators.limitClause - Limit(typedVisit(limit), withOffset) + LimitAll(withOffset) + } else { + withOffset.optional(limit) { + 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 add31448bef78..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,6 +1659,22 @@ 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. + */ +case class LimitAll(child: LogicalPlan) extends UnaryNode { + 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/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..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 @@ -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: Boolean = false) extends LeafNode with MultiInstanceRelation { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) 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 47b5ba365cc20..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) { + val planWithLimit = if (limit.isDefined && limit.get >= 0) { 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] @@ -240,7 +242,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/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 23ee616dda8ea..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 @@ -293,6 +293,108 @@ 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 ++- Project [n#x] + +- SubqueryAlias t + +- 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 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, 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 @@ -315,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 @@ -385,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 @@ -435,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 @@ -514,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 @@ -543,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 @@ -590,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 @@ -637,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 @@ -670,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 @@ -723,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 @@ -796,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 @@ -848,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 @@ -927,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 @@ -950,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 @@ -987,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 @@ -1036,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 @@ -1096,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 @@ -1125,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 @@ -1160,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 @@ -1201,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 @@ -1242,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 @@ -1250,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 @@ -1298,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 @@ -1328,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 @@ -1353,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 @@ -1396,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 @@ -1439,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 @@ -1483,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 @@ -1524,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 @@ -1564,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 @@ -1572,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 @@ -1627,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 @@ -1635,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 @@ -1666,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 @@ -1674,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 @@ -1705,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 @@ -1713,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 @@ -1810,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 @@ -1854,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 @@ -1882,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 @@ -1919,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 @@ -1964,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] @@ -1973,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 @@ -1981,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 @@ -2008,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 @@ -2130,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/resources/sql-tests/inputs/cte-recursion.sql b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql index 828a7c8046776..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 @@ -97,6 +97,37 @@ 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; + +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; + +-- 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 f1fea1ff79eb8..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 @@ -333,6 +333,302 @@ 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 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 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 0c2380a2d90e3..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,6 +786,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession { } test("SPARK-50892: parameterized identifier inside a recursive CTE") { + def query(p: String): String = { s""" |WITH RECURSIVE t1(n) AS (