diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index d52ffc011b727..c563e0b93bef5 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -2157,6 +2157,35 @@ ], "sqlState" : "42602" }, + "INVALID_RECURSIVE_CTE" : { + "message" : [ + "Invalid recursive definition found. Recursive queries must contain an UNION or an UNION ALL statement with 2 children. The first child needs to be the anchor term without any recursive references." + ], + "sqlState" : "42836" + }, + "INVALID_RECURSIVE_REFERENCE" : { + "message" : [ + "Invalid recursive reference found." + ], + "subClass" : { + "DATA_TYPE" : { + "message" : [ + "The data type of recursive references cannot change during resolution. Originally it was but after resolution is ." + ] + }, + "NUMBER" : { + "message" : [ + "Recursive references cannot be used multiple times." + ] + }, + "PLACE" : { + "message" : [ + "Recursive references cannot be used on the right side of left outer/semi/anti joins, on the left side of right outer joins, in full outer joins and in aggregates." + ] + } + }, + "sqlState" : "42836" + }, "INVALID_SCHEMA" : { "message" : [ "The input schema is not a valid schema string." @@ -2898,6 +2927,18 @@ ], "sqlState" : "38000" }, + "RECURSIVE_CTE_IN_LEGACY_MODE" : { + "message" : [ + "Recursive definitions cannot be used in legacy CTE precedence mode (spark.sql.legacy.ctePrecedencePolicy=LEGACY)." + ], + "sqlState" : "42836" + }, + "RECURSIVE_CTE_WHEN_INLINING_IS_FORCED" : { + "message" : [ + "Recursive definitions cannot be used when CTE inlining is forced." + ], + "sqlState" : "42836" + }, "RECURSIVE_PROTOBUF_SCHEMA" : { "message" : [ "Found recursive reference in Protobuf schema, which can not be processed by Spark by default: . try setting the option `recursive.fields.max.depth` 0 to 10. Going beyond 10 levels of recursion is not allowed." diff --git a/docs/sql-error-conditions-invalid-recursive-reference-error-class.md b/docs/sql-error-conditions-invalid-recursive-reference-error-class.md new file mode 100644 index 0000000000000..8532ba207bd42 --- /dev/null +++ b/docs/sql-error-conditions-invalid-recursive-reference-error-class.md @@ -0,0 +1,40 @@ +--- +layout: global +title: INVALID_RECURSIVE_REFERENCE error class +displayTitle: INVALID_RECURSIVE_REFERENCE error class +license: | + 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. +--- + +[SQLSTATE: 42836](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Invalid recursive reference found. + +This error class has the following derived error classes: + +## DATA_TYPE + +The data type of recursive references cannot change during resolution. Originally it was `` but after resolution is ``. + +## NUMBER + +Recursive references cannot be used multiple times. + +## PLACE + +Recursive references cannot be used on the right side of left outer/semi/anti joins, on the left side of right outer joins, in full outer joins and in aggregates. + + diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 82befaae81df3..882eef0fa053b 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -1249,6 +1249,20 @@ For more details see [INVALID_PARTITION_OPERATION](sql-error-conditions-invalid- `` is an invalid property value, please use quotes, e.g. SET ``=`` +### INVALID_RECURSIVE_CTE + +[SQLSTATE: 42836](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Invalid recursive definition found. Recursive queries must contain an UNION or an UNION ALL statement with 2 children. The first child needs to be the anchor term without any recursive references. + +### [INVALID_RECURSIVE_REFERENCE](sql-error-conditions-invalid-recursive-reference-error-class.html) + +[SQLSTATE: 42836](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Invalid recursive reference found. + +For more details see [INVALID_RECURSIVE_REFERENCE](sql-error-conditions-invalid-recursive-reference-error-class.html) + ### [INVALID_SCHEMA](sql-error-conditions-invalid-schema-error-class.html) [SQLSTATE: 42K07](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) @@ -1762,6 +1776,18 @@ Protobuf type not yet supported: ``. Failed to `` Python data source `` in Python: `` +### RECURSIVE_CTE_IN_LEGACY_MODE + +[SQLSTATE: 42836](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Recursive definitions cannot be used in legacy CTE precedence mode (spark.sql.legacy.ctePrecedencePolicy=LEGACY). + +### RECURSIVE_CTE_WHEN_INLINING_IS_FORCED + +[SQLSTATE: 42836](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Recursive definitions cannot be used when CTE inlining is forced. + ### RECURSIVE_PROTOBUF_SCHEMA [SQLSTATE: 42K0G](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 90e65e5ce36ed..2dfbe62ac97f6 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -593,6 +593,7 @@ Below is a list of all the keywords in Spark SQL. |RECORDREADER|non-reserved|non-reserved|non-reserved| |RECORDWRITER|non-reserved|non-reserved|non-reserved| |RECOVER|non-reserved|non-reserved|non-reserved| +|RECURSIVE|reserved|non-reserved|reserved| |REDUCE|non-reserved|non-reserved|non-reserved| |REFERENCES|reserved|non-reserved|reserved| |REFRESH|non-reserved|non-reserved|non-reserved| diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 index 9b3dcbc6d194f..85a1841374694 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 @@ -312,6 +312,7 @@ REAL: 'REAL'; RECORDREADER: 'RECORDREADER'; RECORDWRITER: 'RECORDWRITER'; RECOVER: 'RECOVER'; +RECURSIVE: 'RECURSIVE'; REDUCE: 'REDUCE'; REFERENCES: 'REFERENCES'; REFRESH: 'REFRESH'; diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 0ab3c5ac0c361..703bfd0aaaf64 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -372,7 +372,7 @@ describeColName ; ctes - : WITH namedQuery (COMMA namedQuery)* + : WITH RECURSIVE? namedQuery (COMMA namedQuery)* ; namedQuery @@ -1806,6 +1806,7 @@ nonReserved | RECORDREADER | RECORDWRITER | RECOVER + | RECURSIVE | REDUCE | REFERENCES | REFRESH 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 ec91f9b21a76b..eea56c142aba5 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 @@ -350,7 +350,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor Batch("HandleSpecialCommand", Once, HandleSpecialCommand), Batch("Remove watermark for batch query", Once, - EliminateEventTimeWatermark) + EliminateEventTimeWatermark), + Batch("Insert Loops", Once, + InsertLoops) ) /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala index 173c9d44a2b32..c2abf2fdc68a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -123,7 +123,7 @@ object CTESubstitution extends Rule[LogicalPlan] { startOfQuery: Boolean = true): Unit = { val resolver = conf.resolver plan match { - case UnresolvedWith(child, relations) => + case UnresolvedWith(child, relations, _) => val newNames = ArrayBuffer.empty[String] newNames ++= outerCTERelationNames relations.foreach { @@ -149,10 +149,15 @@ object CTESubstitution extends Rule[LogicalPlan] { plan: LogicalPlan, cteDefs: ArrayBuffer[CTERelationDef]): LogicalPlan = { plan.resolveOperatorsUp { - case cte @ UnresolvedWith(child, relations) => - val resolvedCTERelations = - resolveCTERelations(relations, isLegacy = true, forceInline = false, Seq.empty, cteDefs) - val substituted = substituteCTE(child, alwaysInline = true, resolvedCTERelations) + case cte @ UnresolvedWith(child, relations, allowRecursion) => + if (allowRecursion) { + cte.failAnalysis( + errorClass = "RECURSIVE_CTE_IN_LEGACY_MODE", + messageParameters = Map.empty) + } + val resolvedCTERelations = resolveCTERelations(relations, isLegacy = true, + forceInline = false, Seq.empty, cteDefs, allowRecursion) + val substituted = substituteCTE(child, alwaysInline = true, resolvedCTERelations, None)._1 substituted.copyTagsFrom(cte) substituted } @@ -204,14 +209,20 @@ object CTESubstitution extends Rule[LogicalPlan] { var firstSubstituted: Option[LogicalPlan] = None val newPlan = plan.resolveOperatorsDownWithPruning( _.containsAnyPattern(UNRESOLVED_WITH, PLAN_EXPRESSION)) { - case cte @ UnresolvedWith(child: LogicalPlan, relations) => + case cte @ UnresolvedWith(child, relations, allowRecursion) => + if (allowRecursion && forceInline) { + cte.failAnalysis( + errorClass = "RECURSIVE_CTE_WHEN_INLINING_IS_FORCED", + messageParameters = Map.empty) + } val resolvedCTERelations = - resolveCTERelations(relations, isLegacy = false, forceInline, outerCTEDefs, cteDefs) ++ - outerCTEDefs + resolveCTERelations(relations, isLegacy = false, forceInline, outerCTEDefs, cteDefs, + allowRecursion) ++ outerCTEDefs val substituted = substituteCTE( traverseAndSubstituteCTE(child, forceInline, resolvedCTERelations, cteDefs)._1, forceInline, - resolvedCTERelations) + resolvedCTERelations, + None)._1 if (firstSubstituted.isEmpty) { firstSubstituted = Some(substituted) } @@ -231,7 +242,8 @@ object CTESubstitution extends Rule[LogicalPlan] { isLegacy: Boolean, forceInline: Boolean, outerCTEDefs: Seq[(String, CTERelationDef)], - cteDefs: ArrayBuffer[CTERelationDef]): Seq[(String, CTERelationDef)] = { + cteDefs: ArrayBuffer[CTERelationDef], + allowRecursion: Boolean): Seq[(String, CTERelationDef)] = { val alwaysInline = isLegacy || forceInline var resolvedCTERelations = if (alwaysInline) { Seq.empty @@ -250,22 +262,85 @@ object CTESubstitution extends Rule[LogicalPlan] { // NOTE: we must call `traverseAndSubstituteCTE` before `substituteCTE`, as the relations // in the inner CTE have higher priority over the relations in the outer CTE when resolving // inner CTE relations. For example: - // WITH t1 AS (SELECT 1) - // t2 AS ( - // WITH t1 AS (SELECT 2) - // WITH t3 AS (SELECT * FROM t1) - // ) + // WITH + // t1 AS (SELECT 1), + // t2 AS ( + // WITH + // t1 AS (SELECT 2), + // t3 AS (SELECT * FROM t1) + // SELECT * FROM t1 + // ) + // SELECT * FROM t2 // t3 should resolve the t1 to `SELECT 2` instead of `SELECT 1`. - traverseAndSubstituteCTE(relation, forceInline, resolvedCTERelations, cteDefs)._1 + // + // When recursion allowed: + // - don't add current definition to outer definitions of `traverseAndSubstituteCTE()` to + // prevent recursion inside inner CTEs. + // E.g. the following query will not resolve `t1` within `t2`: + // WITH RECURSIVE + // t1 AS ( + // SELECT 1 AS level + // UNION ( + // WITH t2 AS (SELECT level + 1 FROM t1 WHERE level < 10) + // SELECT * FROM t2 + // ) + // ) + // SELECT * FROM t1 + // - remove definitions that conflict with current relation `name` from outer definitions of + // `traverseAndSubstituteCTE()` to prevent weird resolutions. + // E.g. we don't want to resolve `t1` within `t3` to `SELECT 1`: + // WITH + // t1 AS (SELECT 1), + // t2 AS ( + // WITH RECURSIVE + // t1 AS ( + // SELECT 1 AS level + // UNION ( + // WITH t3 AS (SELECT level + 1 FROM t1 WHERE level < 10) + // SELECT * FROM t3 + // ) + // ) + // SELECT * FROM t1 + // ) + // SELECT * FROM t2 + val nonConflictingCTERelations = if (allowRecursion) { + resolvedCTERelations.filterNot { + case (cteName, cteDef) => cteDef.conf.resolver(cteName, name) + } + } else { + resolvedCTERelations + } + traverseAndSubstituteCTE(relation, forceInline, nonConflictingCTERelations, cteDefs)._1 + } + + // If recursion is allowed then it has higher priority than outer or previous relations so + // construct a not yet substituted but recursive `CTERelationDef`, that we will prepend to + // `resolvedCTERelations`. + val recursiveCTERelation = if (allowRecursion) { + Some(name -> CTERelationDef(relation, recursive = true)) + } else { + None } - // CTE definition can reference a previous one - val substituted = substituteCTE(innerCTEResolved, alwaysInline, resolvedCTERelations) - val cteRelation = CTERelationDef(substituted) + + // CTE definition can reference a previous one or itself if recursion allowed. + val (substituted, recursionFound) = substituteCTE(innerCTEResolved, alwaysInline, + resolvedCTERelations, recursiveCTERelation) + val cteRelation = recursiveCTERelation + .map(_._2.copy(child = substituted, recursive = recursionFound)) + .getOrElse(CTERelationDef(substituted)) if (!alwaysInline) { cteDefs += cteRelation } + + // From this point any reference to the definition is non-recursive. + val nonRecursiveCTERelation = if (cteRelation.recursive) { + cteRelation.copy(recursive = false) + } else { + cteRelation + } + // Prepending new CTEs makes sure that those have higher priority over outer ones. - resolvedCTERelations +:= (name -> cteRelation) + resolvedCTERelations +:= (name -> nonRecursiveCTERelation) } resolvedCTERelations } @@ -273,30 +348,41 @@ object CTESubstitution extends Rule[LogicalPlan] { private def substituteCTE( plan: LogicalPlan, alwaysInline: Boolean, - cteRelations: Seq[(String, CTERelationDef)]): LogicalPlan = { - plan.resolveOperatorsUpWithPruning( + cteRelations: Seq[(String, CTERelationDef)], + recursiveCTERelation: Option[(String, CTERelationDef)]): (LogicalPlan, Boolean) = { + var recursionFound = false + val substituted = plan.resolveOperatorsUpWithPruning( _.containsAnyPattern(RELATION_TIME_TRAVEL, UNRESOLVED_RELATION, PLAN_EXPRESSION)) { case RelationTimeTravel(UnresolvedRelation(Seq(table), _, _), _, _) if cteRelations.exists(r => plan.conf.resolver(r._1, table)) => throw QueryCompilationErrors.timeTravelUnsupportedError(toSQLId(table)) case u @ UnresolvedRelation(Seq(table), _, _) => - cteRelations.find(r => plan.conf.resolver(r._1, table)).map { case (_, d) => - if (alwaysInline) { - d.child - } else { - // Add a `SubqueryAlias` for hint-resolving rules to match relation names. - SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output, d.isStreaming)) - } + (recursiveCTERelation ++ cteRelations) + .find(r => plan.conf.resolver(r._1, table)) + .map { case (_, d) => + if (alwaysInline) { + d.child + } else { + if (d.recursive) { + recursionFound = true + } + // Add a `SubqueryAlias` for hint-resolving rules to match relation names. + SubqueryAlias(table, + CTERelationRef(d.id, d.resolved, d.output, d.isStreaming, recursive = d.recursive)) + } }.getOrElse(u) case other => // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. other.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { case e: SubqueryExpression => - e.withNewPlan(apply(substituteCTE(e.plan, alwaysInline, cteRelations))) + e.withNewPlan( + apply(substituteCTE(e.plan, alwaysInline, cteRelations, None)._1)) } } + + (substituted, recursionFound) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 1ce984a39b277..a85e18033c3d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -780,6 +780,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case _ => // Analysis successful! } } + checkRecursion(plan) checkCollectedMetrics(plan) extendedCheckRules.foreach(_(plan)) plan.foreachUp { @@ -897,6 +898,73 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB if (Utils.isTesting) scrubOutIds(result) else result } + /** + * Recursion according to SQL standard comes with several limitations due to the fact that only + * those operations are allowed where the new set of rows can be computed from the result of the + * previous iteration. This implies that a recursive reference can't be used in some kinds of + * joins and aggregations. + * A further constraint is that a recursive term can contain one recursive reference only. + * + * This rule checks that these restrictions are not violated. + */ + private def checkRecursion( + plan: LogicalPlan, + references: mutable.Map[Long, (Int, Seq[DataType])] = mutable.Map.empty): Unit = { + plan match { + case UnionLoop(id, anchor, recursion, _) => + checkRecursion(anchor, references) + checkRecursion(recursion, references += id -> (0, anchor.output.map(_.dataType))) + references -= id + case r @ UnionLoopRef(loopId, output, false) => + if (!references.contains(loopId)) { + r.failAnalysis( + errorClass = "INVALID_RECURSIVE_REFERENCE.PLACE", + messageParameters = Map.empty + ) + } + val (count, dataType) = references(loopId) + if (count > 0) { + r.failAnalysis( + errorClass = "INVALID_RECURSIVE_REFERENCE.NUMBER", + messageParameters = Map.empty + ) + } + val originalDataType = r.output.map(_.dataType) + if (!originalDataType.zip(dataType).forall { + case (odt, dt) => DataType.equalsStructurally(odt, dt, true) + }) { + r.failAnalysis( + errorClass = "INVALID_RECURSIVE_REFERENCE.DATA_TYPE", + messageParameters = Map( + "fromDataType" -> originalDataType.map(toSQLType).mkString(", "), + "toDataType" -> dataType.map(toSQLType).mkString(", ") + ) + ) + } + references(loopId) = (count + 1, dataType) + case Join(left, right, Inner, _, _) => + checkRecursion(left, references) + checkRecursion(right, references) + case Join(left, right, LeftOuter, _, _) => + checkRecursion(left, references) + checkRecursion(right, mutable.Map.empty) + case Join(left, right, RightOuter, _, _) => + checkRecursion(left, mutable.Map.empty) + checkRecursion(right, references) + case Join(left, right, LeftSemi, _, _) => + checkRecursion(left, references) + checkRecursion(right, mutable.Map.empty) + case Join(left, right, LeftAnti, _, _) => + checkRecursion(left, references) + checkRecursion(right, mutable.Map.empty) + case Join(left, right, _, _, _) => + checkRecursion(left, mutable.Map.empty) + checkRecursion(right, mutable.Map.empty) + case Aggregate(_, _, child) => checkRecursion(child, mutable.Map.empty) + case o => o.children.foreach(checkRecursion(_, references)) + } + } + /** * Validates subquery expressions in the plan. Upon failure, returns an user facing error. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/InsertLoops.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/InsertLoops.scala new file mode 100644 index 0000000000000..2653d24ce29a7 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/InsertLoops.scala @@ -0,0 +1,89 @@ +/* + * 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._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.CTE + +/** + * This rule transforms recursive [[Union]] nodes into [[UnionLoop]] and recursive + * [[CTERelationRef]] nodes into [[UnionLoopRef]] nodes in recursive CTE definitions. + */ +case object InsertLoops extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDownWithPruning( + _.containsPattern(CTE)) { + case w: WithCTE => + val newCTEDefs = w.cteDefs.map { + case cte if cte.recursive => + + def transformRefs(plan: LogicalPlan, accumulated: Boolean) = { + plan.transformWithPruning(_.containsPattern(CTE)) { + case r: CTERelationRef if r.recursive => + UnionLoopRef(r.cteId, r.output, accumulated) + } + } + + cte.child match { + case Union(Seq(anchor, recursion), false, false) => + cte.copy(child = UnionLoop(cte.id, anchor, transformRefs(recursion, false))) + case a @ SubqueryAlias(_, Union(Seq(anchor, recursion), false, false)) => + cte.copy(child = + a.copy(child = + UnionLoop(cte.id, anchor, transformRefs(recursion, false)))) + case p @ Project(_, Union(Seq(anchor, recursion), false, false)) => + cte.copy(child = + p.copy(child = + UnionLoop(cte.id, anchor, transformRefs(recursion, false)))) + case a @ SubqueryAlias(_, + p @ Project(_, Union(Seq(anchor, recursion), false, false))) => + cte.copy(child = + a.copy(child = + p.copy(child = + UnionLoop(cte.id, anchor, transformRefs(recursion, false))))) + + // If the recursion is described with an UNION clause then the recursive term should + // not return those rows that have been calculated previously so we exclude those rows + // from the current iteration result. + case p @ Project(_, Distinct(Union(Seq(anchor, recursion), false, false))) => + cte.copy(child = + p.copy(child = + UnionLoop(cte.id, + Distinct(anchor), + Except( + transformRefs(recursion, false), + UnionLoopRef(cte.id, cte.output, true), + false)))) + case a @ SubqueryAlias(_, + p @ Project(_, Distinct(Union(Seq(anchor, recursion), false, false)))) => + cte.copy(child = + a.copy(child = + p.copy(child = + UnionLoop(cte.id, + Distinct(anchor), + Except( + transformRefs(recursion, false), + UnionLoopRef(cte.id, cte.output, true), + false))))) + case _ => cte + } + case o => o + } + w.copy(cteDefs = newCTEDefs) + } +} 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 f1077378b2d9c..672770b588677 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable import org.apache.spark.sql.catalyst.expressions.SubqueryExpression -import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, LogicalPlan, WithCTE} +import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, Distinct, LogicalPlan, SubqueryAlias, Union, WithCTE} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{CTE, PLAN_EXPRESSION} @@ -37,21 +37,89 @@ object ResolveWithCTE extends Rule[LogicalPlan] { } } + private def updateRecursiveAnchor(cteDef: CTERelationDef): CTERelationDef = { + cteDef.child match { + case SubqueryAlias(_, u: Union) => + if (u.children.head.resolved) { + cteDef.copy(recursionAnchor = Some(u.children.head)) + } else { + cteDef + } + case SubqueryAlias(_, d @ Distinct(u: Union)) => + if (u.children.head.resolved) { + cteDef.copy(recursionAnchor = Some(d.copy(child = u.children.head))) + } else { + cteDef + } + case SubqueryAlias(_, a @ UnresolvedSubqueryColumnAliases(_, u: Union)) => + if (u.children.head.resolved) { + cteDef.copy(recursionAnchor = Some(a.copy(child = u.children.head))) + } else { + cteDef + } + case SubqueryAlias(_, a @ UnresolvedSubqueryColumnAliases(_, d @ Distinct(u: Union))) => + if (u.children.head.resolved) { + cteDef.copy(recursionAnchor = Some(a.copy(child = d.copy(child = u.children.head)))) + } else { + cteDef + } + case _ => + cteDef.failAnalysis( + errorClass = "INVALID_RECURSIVE_CTE", + messageParameters = Map.empty) + } + } + private def resolveWithCTE( plan: LogicalPlan, cteDefMap: mutable.HashMap[Long, CTERelationDef]): LogicalPlan = { plan.resolveOperatorsDownWithPruning(_.containsAllPatterns(CTE)) { case w @ WithCTE(_, cteDefs) => - cteDefs.foreach { cteDef => - if (cteDef.resolved) { - cteDefMap.put(cteDef.id, cteDef) + val newCTEDefs = cteDefs.map { cteDef => + // If a recursive CTE definition is not yet resolved then extract the anchor term to the + // definition, but if it is resolved then the extracted anchor term is no longer needed + // and can be removed. + val newCTEDef = if (cteDef.recursive) { + if (!cteDef.resolved) { + if (cteDef.recursionAnchor.isEmpty) { + updateRecursiveAnchor(cteDef) + } else { + cteDef + } + } else { + if (cteDef.recursionAnchor.nonEmpty) { + cteDef.copy(recursionAnchor = None) + } else { + cteDef + } + } + } else { + cteDef + } + + if (newCTEDef.resolved || newCTEDef.recursionAnchorResolved) { + cteDefMap.put(newCTEDef.id, newCTEDef) } + + newCTEDef } - w + w.copy(cteDefs = newCTEDefs) case ref: CTERelationRef if !ref.resolved => cteDefMap.get(ref.cteId).map { cteDef => - CTERelationRef(cteDef.id, cteDef.resolved, cteDef.output, cteDef.isStreaming) + // Recursive references can be resolved from the anchor term. + if (ref.recursive) { + if (cteDef.recursionAnchorResolved) { + ref.copy(_resolved = true, output = cteDef.recursionAnchor.get.output, + isStreaming = cteDef.isStreaming) + } else { + ref + } + } else if (cteDef.resolved) { + ref.copy(_resolved = true, output = cteDef.output, isStreaming = cteDef.isStreaming) + } else { + ref + } }.getOrElse { ref } 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 8d7ff4cbf163d..9b3c5f5c554bf 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 @@ -65,7 +65,8 @@ case class InlineCTE(alwaysInline: Boolean = false) extends Rule[LogicalPlan] { // 1) It is fine to inline a CTE if it references another CTE that is non-deterministic; // 2) Any `CTERelationRef` that contains `OuterReference` would have been inlined first. refCount == 1 || - cteDef.deterministic || + // Don't inline recursive CTEs if not necessary as recursion is very costly. + (cteDef.deterministic && !cteDef.recursive) || cteDef.child.exists(_.expressions.exists(_.isInstanceOf[OuterReference])) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index a4b25cbd1d2ee..ef454b22729ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -155,13 +155,12 @@ abstract class Optimizer(catalogManager: CatalogManager) // Optimizer rules start here ////////////////////////////////////////////////////////////////////////////////////////// Batch("Eliminate Distinct", Once, EliminateDistinct) :: + Batch("Inline CTE", Once, InlineCTE()) :: // - Do the first call of CombineUnions before starting the major Optimizer rules, // since it can reduce the number of iteration and the other rules could add/move // extra operators between two adjacent Union operators. // - Call CombineUnions again in Batch("Operator Optimizations"), // since the other rules might make two separate Unions operators adjacent. - Batch("Inline CTE", Once, - InlineCTE()) :: Batch("Union", fixedPoint, RemoveNoopOperators, CombineUnions, @@ -750,6 +749,11 @@ object LimitPushDown extends Rule[LogicalPlan] { case LocalLimit(exp, u: Union) => LocalLimit(exp, u.copy(children = u.children.map(maybePushLocalLimit(exp, _)))) + case l @ LocalLimit(IntegerLiteral(limit), p @ Project(_, u: UnionLoop)) => + l.copy(child = p.copy(child = u.copy(limit = Some(limit)))) + case l @ LocalLimit(IntegerLiteral(limit), u: UnionLoop) => + l.copy(child = u.copy(limit = Some(limit))) + // Add extra limits below JOIN: // 1. For LEFT OUTER and RIGHT OUTER JOIN, we push limits to the left and right sides // respectively if join condition is not empty. @@ -933,6 +937,9 @@ object ColumnPruning extends Rule[LogicalPlan] { } else { p } + // TODO: Pruning `UnionLoop`s needs to take into account both the outer `Project` and the inner + // `UnionLoopRef` nodes. + case p @ Project(_, _: UnionLoop) => p // Prune unnecessary window expressions case p @ Project(_, w: Window) if !w.windowOutputSet.subsetOf(p.references) => 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 aa13e6a67c510..838827a0e2e1e 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 @@ -122,7 +122,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { private def pushdownPredicatesAndAttributes( plan: LogicalPlan, cteMap: CTEMap): LogicalPlan = plan.transformWithSubqueries { - case cteDef @ CTERelationDef(child, id, originalPlanWithPredicates, _) => + case cteDef @ CTERelationDef(child, id, originalPlanWithPredicates, _, _, _) => val (_, _, newPreds, newAttrSet) = cteMap(id) val originalPlan = originalPlanWithPredicates.map(_._1).getOrElse(child) val preds = originalPlanWithPredicates.map(_._2).getOrElse(Seq.empty) @@ -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) @@ -170,7 +170,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { object CleanUpTempCTEInfo extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(CTE)) { - case cteDef @ CTERelationDef(_, _, Some(_), _) => + case cteDef @ CTERelationDef(_, _, Some(_), _, _, _) => cteDef.copy(originalPlanWithPredicates = None) } } 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 9a4e389fcf99d..eb186a0376543 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 @@ -183,7 +183,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { throw QueryParsingErrors.duplicateCteDefinitionNamesError( duplicates.mkString("'", "', '", "'"), ctx) } - UnresolvedWith(plan, ctes.toSeq) + UnresolvedWith(plan, ctes.toSeq, ctx.RECURSIVE() != null) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index ef7cd7401f25e..7f3fa72fcf6d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -695,4 +695,15 @@ object QueryPlan extends PredicateHelper { case e: AnalysisException => append(e.toString) } } + + /** + * Generate detailed field string with different format based on type of input value + */ + def generateFieldString(fieldName: String, values: Any): String = values match { + case iter: Iterable[_] if (iter.size == 0) => s"${fieldName}: []" + case iter: Iterable[_] => s"${fieldName} [${iter.size}]: ${iter.mkString("[", ", ", "]")}" + case str: String if (str == null || str.isEmpty) => s"${fieldName}: None" + case str: String => s"${fieldName}: ${str}" + case _ => throw new IllegalArgumentException(s"Unsupported type for argument values: $values") + } } 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 65f4151c0c963..5c71b09bc61bb 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 @@ -428,6 +428,61 @@ object Union { } } +abstract class UnionBase extends LogicalPlan { + // updating nullability to make all the children consistent + override def output: Seq[Attribute] = { + children.map(_.output).transpose.map { attrs => + val firstAttr = attrs.head + val nullable = attrs.exists(_.nullable) + val newDt = attrs.map(_.dataType).reduce(StructType.unionLikeMerge) + if (firstAttr.dataType == newDt) { + firstAttr.withNullability(nullable) + } else { + AttributeReference(firstAttr.name, newDt, nullable, firstAttr.metadata)( + firstAttr.exprId, firstAttr.qualifier) + } + } + } + + override def metadataOutput: Seq[Attribute] = Nil + + /** + * Maps the constraints containing a given (original) sequence of attributes to those with a + * given (reference) sequence of attributes. Given the nature of union, we expect that the + * mapping between the original and reference sequences are symmetric. + */ + private def rewriteConstraints( + reference: Seq[Attribute], + original: Seq[Attribute], + constraints: ExpressionSet): ExpressionSet = { + require(reference.size == original.size) + val attributeRewrites = AttributeMap(original.zip(reference)) + constraints.map(_ transform { + case a: Attribute => attributeRewrites(a) + }) + } + + private def merge(a: ExpressionSet, b: ExpressionSet): ExpressionSet = { + val common = a.intersect(b) + // The constraint with only one reference could be easily inferred as predicate + // Grouping the constraints by it's references so we can combine the constraints with same + // reference together + val othera = a.diff(common).filter(_.references.size == 1).groupBy(_.references.head) + val otherb = b.diff(common).filter(_.references.size == 1).groupBy(_.references.head) + // loose the constraints by: A1 && B1 || A2 && B2 -> (A1 || A2) && (B1 || B2) + val others = (othera.keySet intersect otherb.keySet).map { attr => + Or(othera(attr).reduceLeft(And), otherb(attr).reduceLeft(And)) + } + common ++ others + } + + override protected lazy val validConstraints: ExpressionSet = { + children + .map(child => rewriteConstraints(children.head.output, child.output, child.constraints)) + .reduce(merge(_, _)) + } +} + /** * Logical plan for unioning multiple plans, without a distinct. This is UNION ALL in SQL. * @@ -439,7 +494,7 @@ object Union { case class Union( children: Seq[LogicalPlan], byName: Boolean = false, - allowMissingCol: Boolean = false) extends LogicalPlan { + allowMissingCol: Boolean = false) extends UnionBase { assert(!allowMissingCol || byName, "`allowMissingCol` can be true only if `byName` is true.") override def maxRows: Option[Long] = { @@ -482,23 +537,6 @@ case class Union( AttributeSet.fromAttributeSets(children.map(_.outputSet)).size } - // updating nullability to make all the children consistent - override def output: Seq[Attribute] = { - children.map(_.output).transpose.map { attrs => - val firstAttr = attrs.head - val nullable = attrs.exists(_.nullable) - val newDt = attrs.map(_.dataType).reduce(StructType.unionLikeMerge) - if (firstAttr.dataType == newDt) { - firstAttr.withNullability(nullable) - } else { - AttributeReference(firstAttr.name, newDt, nullable, firstAttr.metadata)( - firstAttr.exprId, firstAttr.qualifier) - } - } - } - - override def metadataOutput: Seq[Attribute] = Nil - override lazy val resolved: Boolean = { // allChildrenCompatible needs to be evaluated after childrenResolved def allChildrenCompatible: Boolean = @@ -512,44 +550,58 @@ case class Union( children.length > 1 && !(byName || allowMissingCol) && childrenResolved && allChildrenCompatible } - /** - * Maps the constraints containing a given (original) sequence of attributes to those with a - * given (reference) sequence of attributes. Given the nature of union, we expect that the - * mapping between the original and reference sequences are symmetric. - */ - private def rewriteConstraints( - reference: Seq[Attribute], - original: Seq[Attribute], - constraints: ExpressionSet): ExpressionSet = { - require(reference.size == original.size) - val attributeRewrites = AttributeMap(original.zip(reference)) - constraints.map(_ transform { - case a: Attribute => attributeRewrites(a) - }) - } + override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): Union = + copy(children = newChildren) +} - private def merge(a: ExpressionSet, b: ExpressionSet): ExpressionSet = { - val common = a.intersect(b) - // The constraint with only one reference could be easily inferred as predicate - // Grouping the constraints by it's references so we can combine the constraints with same - // reference together - val othera = a.diff(common).filter(_.references.size == 1).groupBy(_.references.head) - val otherb = b.diff(common).filter(_.references.size == 1).groupBy(_.references.head) - // loose the constraints by: A1 && B1 || A2 && B2 -> (A1 || A2) && (B1 || B2) - val others = (othera.keySet intersect otherb.keySet).map { attr => - Or(othera(attr).reduceLeft(And), otherb(attr).reduceLeft(And)) - } - common ++ others - } +/** + * The logical node for recursion, that contains a initial (anchor) and a recursion describing term, + * that contains an [[UnionLoopRef]] node. + * The node is very similar to [[Union]] because the initial and "generated" children are union-ed + * and it is also similar to a loop because the recursion continues until the last generated child + * is not empty. + * + * @param id The id of the loop, inherited from [[CTERelationDef]] + * @param anchor The plan of the initial element of the loop. + * @param recursion The plan that describes the recursion with an [[UnionLoopRef]] node. + * @param limit An optional limit that can be pushed down to the node to stop the loop earlier. + */ +case class UnionLoop( + id: Long, + anchor: LogicalPlan, + recursion: LogicalPlan, + limit: Option[Int] = None) extends UnionBase { + override def children: Seq[LogicalPlan] = Seq(anchor, recursion) + + override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): UnionLoop = + copy(anchor = newChildren(0), recursion = newChildren(1)) +} - override protected lazy val validConstraints: ExpressionSet = { - children - .map(child => rewriteConstraints(children.head.output, child.output, child.constraints)) - .reduce(merge(_, _)) - } +/** + * The recursive reference in the recursive term of an [[UnionLoop]] node. + * + * @param loopId The id of the loop, inherited from [[CTERelationRef]] + * @param output The output attributes of this recursive reference. + * @param accumulated If false the the reference stands for the result of the previous iteration. + * If it is true then then it stands for the union of all previous iteration + * results. + */ +case class UnionLoopRef( + loopId: Long, + override val output: Seq[Attribute], + accumulated: Boolean) extends LeafNode with MultiInstanceRelation { + override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) - override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): Union = - copy(children = newChildren) + override def computeStats(): Statistics = Statistics(SQLConf.get.defaultSizeInBytes) + + override def verboseStringWithOperatorId(): String = { + s""" + |$formattedNodeName + |Loop id: $loopId + |${QueryPlan.generateFieldString("Output", output)} + |Accumulated: $accumulated + |""".stripMargin + } } case class Join( @@ -803,10 +855,12 @@ object View { * @param child The final query of this CTE. * @param cteRelations A sequence of pair (alias, the CTE definition) that this CTE defined * Each CTE can see the base tables and the previously defined CTEs only. + * @param allowRecursion A boolean flag if recursion is allowed. */ case class UnresolvedWith( child: LogicalPlan, - cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { + cteRelations: Seq[(String, SubqueryAlias)], + allowRecursion: Boolean = false) extends UnaryNode { final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_WITH) override def output: Seq[Attribute] = child.output @@ -832,19 +886,31 @@ case class UnresolvedWith( * pushdown to help ensure rule idempotency. * @param underSubquery If true, it means we don't need to add a shuffle for this CTE relation as * subquery reuse will be applied to reuse CTE relation output. + * @param recursive If true the definition is recursive. + * @param recursionAnchor A helper plan node that temporary stores the anchor term of recursive + * definitions. In the beginning of recursive resolution the `ResolveWithCTE` + * rule updates this parameter and once it is resolved the same rule resolves + * the recursive [[CTERelationRef]] references and removed this parameter. */ case class CTERelationDef( child: LogicalPlan, id: Long = CTERelationDef.newId, originalPlanWithPredicates: Option[(LogicalPlan, Seq[Expression])] = None, - underSubquery: Boolean = false) extends UnaryNode { + underSubquery: Boolean = false, + recursive: Boolean = false, + recursionAnchor: Option[LogicalPlan] = None) extends LogicalPlan { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) - override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = - copy(child = newChild) + override def children: Seq[LogicalPlan] = child +: recursionAnchor.toSeq + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): CTERelationDef = + copy(child = newChildren.head, recursionAnchor = newChildren.tail.headOption) override def output: Seq[Attribute] = if (resolved) child.output else Nil + + lazy val recursionAnchorResolved = recursionAnchor.map(_.resolved).getOrElse(false) } object CTERelationDef { @@ -861,13 +927,15 @@ object CTERelationDef { * de-duplication. * @param statsOpt The optional statistics inferred from the corresponding CTE * definition. + * @param recursive If this is a recursive reference. */ case class CTERelationRef( cteId: Long, _resolved: Boolean, override val output: Seq[Attribute], override val isStreaming: Boolean, - statsOpt: Option[Statistics] = None) extends LeafNode with MultiInstanceRelation { + statsOpt: Option[Statistics] = None, + recursive: Boolean = false) extends LeafNode with MultiInstanceRelation { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9918d583d49e9..7a3697d80371a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3889,6 +3889,31 @@ object SQLConf { .checkValues(LegacyBehaviorPolicy.values.map(_.toString)) .createWithDefault(LegacyBehaviorPolicy.CORRECTED.toString) + val CTE_RECURSION_LEVEL_LIMIT = buildConf("spark.sql.cteRecursionLevelLimit") + .internal() + .doc("Maximum level of recursion that is allowed wile executing a recursive CTE definition." + + "If a query does not get exhausted before reaching this limit it fails. Use -1 for " + + "unlimited.") + .version("4.0.0") + .intConf + .createWithDefault(100) + + object CTERecursionCacheMode extends Enumeration { + val NONE, REPARTITION, PERSIST, LOCAL_CHECKPOINT, CHECKPOINT = Value + } + + val CTE_RECURSION_CACHE_MODE = buildConf("spark.sql.cteRecursionCacheMode") + .internal() + .doc("The way that partial results of recursive iterations are cached. Caching the partial " + + "result of a recursive iteration is useful so as to calculate the result of the next " + + "iteration and to calculate the union of partial results (the final result of the " + + "recursion).") + .version("4.0.0") + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .checkValues(CTERecursionCacheMode.values.map(_.toString)) + .createWithDefault(CTERecursionCacheMode.REPARTITION.toString) + val LEGACY_INLINE_CTE_IN_COMMANDS = buildConf("spark.sql.legacy.inlineCTEInCommands") .internal() .doc("If true, always inline the CTE relations for the queries in commands. This is the " + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 17dd7349e7bea..ee3631c596854 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -49,6 +49,7 @@ class PlanParserSuite extends AnalysisTest { private def cte( plan: LogicalPlan, + allowRecursion: Boolean, namedPlans: (String, (LogicalPlan, Seq[String]))*): UnresolvedWith = { val ctes = namedPlans.map { case (name, (cte, columnAliases)) => @@ -59,7 +60,7 @@ class PlanParserSuite extends AnalysisTest { } name -> SubqueryAlias(name, subquery) } - UnresolvedWith(plan, ctes) + UnresolvedWith(plan, ctes, allowRecursion) } test("single comment case one") { @@ -274,13 +275,13 @@ class PlanParserSuite extends AnalysisTest { test("common table expressions") { assertEqual( "with cte1 as (select * from a) select * from cte1", - cte(table("cte1").select(star()), "cte1" -> ((table("a").select(star()), Seq.empty)))) + cte(table("cte1").select(star()), false, "cte1" -> ((table("a").select(star()), Seq.empty)))) assertEqual( "with cte1 (select 1) select * from cte1", - cte(table("cte1").select(star()), "cte1" -> ((OneRowRelation().select(1), Seq.empty)))) + cte(table("cte1").select(star()), false, "cte1" -> ((OneRowRelation().select(1), Seq.empty)))) assertEqual( "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2", - cte(table("cte2").select(star()), + cte(table("cte2").select(star()), false, "cte1" -> ((OneRowRelation().select(1), Seq.empty)), "cte2" -> ((table("cte1").select(star()), Seq.empty)))) val sql = "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1" @@ -1393,7 +1394,7 @@ class PlanParserSuite extends AnalysisTest { |WITH cte1 AS (SELECT * FROM testcat.db.tab) |SELECT * FROM cte1 """.stripMargin, - cte(table("cte1").select(star()), + cte(table("cte1").select(star()), false, "cte1" -> ((table("testcat", "db", "tab").select(star()), Seq.empty)))) assertEqual( @@ -1404,7 +1405,21 @@ class PlanParserSuite extends AnalysisTest { test("CTE with column alias") { assertEqual( "WITH t(x) AS (SELECT c FROM a) SELECT * FROM t", - cte(table("t").select(star()), "t" -> ((table("a").select($"c"), Seq("x"))))) + cte(table("t").select(star()), false, "t" -> ((table("a").select($"c"), Seq("x"))))) + } + + test("Recursive CTE") { + assertEqual( + """WITH RECURSIVE r(level) AS ( + | SELECT level FROM t + | UNION ALL + | SELECT level + 1 FROM r WHERE level < 9 + |) + |SELECT * FROM r""".stripMargin, + cte(table("r").select(star()), true, + "r" -> ( + table("t").select($"level").union(table("r").where($"level" < 9).select($"level" + 1)), + Seq("level")))) } test("statement containing terminal semicolons") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index e90a956ab4fde..5baaac91312e4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -142,6 +142,11 @@ trait PlanTestBase extends PredicateHelper with SQLHelper with SQLConfHelper { s }.asInstanceOf[Seq[NamedExpression]] Project(projList, child) case c: KeepAnalyzedQuery => c.storeAnalyzedQuery() + case w @ UnresolvedWith(_, cteRelations, _) => + w.copy(cteRelations = cteRelations.map { + case (cteName, ctePlan) => + cteName -> normalizePlan(normalizeExprIds(ctePlan)).asInstanceOf[SubqueryAlias] + }) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index b3b2b0eab0555..125afb870ada3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -80,7 +80,7 @@ trait DataSourceScanExec extends LeafExecNode { s""" |$formattedNodeName - |${ExplainUtils.generateFieldString("Output", output)} + |${QueryPlan.generateFieldString("Output", output)} |${metadataStr.mkString("\n")} |""".stripMargin } @@ -464,7 +464,7 @@ trait FileSourceScanLike extends DataSourceScanExec { s""" |$formattedNodeName - |${ExplainUtils.generateFieldString("Output", output)} + |${QueryPlan.generateFieldString("Output", output)} |${metadataStr.mkString("\n")} |""".stripMargin } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala index 3da3e646f36b0..58035943a15cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala @@ -287,17 +287,6 @@ object ExplainUtils extends AdaptiveSparkPlanHelper { } } - /** - * Generate detailed field string with different format based on type of input value - */ - def generateFieldString(fieldName: String, values: Any): String = values match { - case iter: Iterable[_] if (iter.size == 0) => s"${fieldName}: []" - case iter: Iterable[_] => s"${fieldName} [${iter.size}]: ${iter.mkString("[", ", ", "]")}" - case str: String if (str == null || str.isEmpty) => s"${fieldName}: None" - case str: String => s"${fieldName}: ${str}" - case _ => throw new IllegalArgumentException(s"Unsupported type for argument values: $values") - } - /** * Given a input plan, returns an array of tuples comprising of : * 1. Hosting operator id. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index c65d1931dd1ba..2806c464816f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -577,7 +577,7 @@ trait LeafExecNode extends SparkPlan with LeafLike[SparkPlan] { override def producedAttributes: AttributeSet = outputSet override def verboseStringWithOperatorId(): String = { val argumentString = argString(conf.maxToStringFields) - val outputStr = s"${ExplainUtils.generateFieldString("Output", output)}" + val outputStr = s"${QueryPlan.generateFieldString("Output", output)}" if (argumentString.nonEmpty) { s""" @@ -605,7 +605,7 @@ trait UnaryExecNode extends SparkPlan with UnaryLike[SparkPlan] { override def verboseStringWithOperatorId(): String = { val argumentString = argString(conf.maxToStringFields) - val inputStr = s"${ExplainUtils.generateFieldString("Input", child.output)}" + val inputStr = s"${QueryPlan.generateFieldString("Input", child.output)}" if (argumentString.nonEmpty) { s""" @@ -626,8 +626,8 @@ trait BinaryExecNode extends SparkPlan with BinaryLike[SparkPlan] { override def verboseStringWithOperatorId(): String = { val argumentString = argString(conf.maxToStringFields) - val leftOutputStr = s"${ExplainUtils.generateFieldString("Left output", left.output)}" - val rightOutputStr = s"${ExplainUtils.generateFieldString("Right output", right.output)}" + val leftOutputStr = s"${QueryPlan.generateFieldString("Left output", left.output)}" + val rightOutputStr = s"${QueryPlan.generateFieldString("Right output", right.output)}" if (argumentString.nonEmpty) { s""" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 2d24f997d105e..6fcddecffce53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -909,6 +909,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { GlobalLimitExec(child = planLater(child), offset = offset) :: Nil case union: logical.Union => execution.UnionExec(union.children.map(planLater)) :: Nil + case u @ logical.UnionLoop(id, anchor, recursion, limit) => + execution.UnionLoopExec(id, anchor, recursion, u.output, limit):: Nil case g @ logical.Generate(generator, _, outer, _, _, child) => execution.GenerateExec( generator, g.requiredChildOutput, outer, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala index 5391d5807597c..c4ab67427bfa6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.execution.aggregate import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, NamedExpression} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Final, PartialMerge} +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{ExplainUtils, PartitioningPreservingUnaryExecNode, UnaryExecNode} +import org.apache.spark.sql.execution.{PartitioningPreservingUnaryExecNode, UnaryExecNode} import org.apache.spark.sql.execution.streaming.StatefulOperatorPartitioning /** @@ -40,11 +41,11 @@ trait BaseAggregateExec extends UnaryExecNode with PartitioningPreservingUnaryEx override def verboseStringWithOperatorId(): String = { s""" |$formattedNodeName - |${ExplainUtils.generateFieldString("Input", child.output)} - |${ExplainUtils.generateFieldString("Keys", groupingExpressions)} - |${ExplainUtils.generateFieldString("Functions", aggregateExpressions)} - |${ExplainUtils.generateFieldString("Aggregate Attributes", aggregateAttributes)} - |${ExplainUtils.generateFieldString("Results", resultExpressions)} + |${QueryPlan.generateFieldString("Input", child.output)} + |${QueryPlan.generateFieldString("Keys", groupingExpressions)} + |${QueryPlan.generateFieldString("Functions", aggregateExpressions)} + |${QueryPlan.generateFieldString("Aggregate Attributes", aggregateAttributes)} + |${QueryPlan.generateFieldString("Results", resultExpressions)} |""".stripMargin } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 083858e4fe80a..824dce4b240c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -26,13 +26,17 @@ import scala.concurrent.duration.Duration import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, SparkException, TaskContext} import org.apache.spark.rdd.{EmptyRDD, PartitionwiseSampledRDD, RDD} +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical.{Limit, LogicalPlan, Project, Union, UnionLoopRef} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.internal.SQLConf.CTERecursionCacheMode import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.ThreadUtils @@ -109,8 +113,8 @@ case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) override def verboseStringWithOperatorId(): String = { s""" |$formattedNodeName - |${ExplainUtils.generateFieldString("Output", projectList)} - |${ExplainUtils.generateFieldString("Input", child.output)} + |${QueryPlan.generateFieldString("Output", projectList)} + |${QueryPlan.generateFieldString("Input", child.output)} |""".stripMargin } @@ -291,7 +295,7 @@ case class FilterExec(condition: Expression, child: SparkPlan) override def verboseStringWithOperatorId(): String = { s""" |$formattedNodeName - |${ExplainUtils.generateFieldString("Input", child.output)} + |${QueryPlan.generateFieldString("Input", child.output)} |Condition : ${condition} |""".stripMargin } @@ -714,6 +718,135 @@ case class UnionExec(children: Seq[SparkPlan]) extends SparkPlan { copy(children = newChildren) } +/** + * The physical node for recursion. + * + * @param loopId The id of the loop. + * @param anchor The logical plan of the initial element of the loop. + * @param recursion The logical plan that describes the recursion with an [[UnionLoopRef]] node. + * @param output The output attributes of this loop. + * @param limit An optional limit that can be pushed down to the node to stop the loop earlier. + */ +case class UnionLoopExec( + loopId: Long, + @transient anchor: LogicalPlan, + @transient recursion: LogicalPlan, + override val output: Seq[Attribute], + limit: Option[Int] = None) extends LeafExecNode { + + val levelLimit = conf.getConf(SQLConf.CTE_RECURSION_LEVEL_LIMIT) + val cacheMode = CTERecursionCacheMode.withName(conf.getConf(SQLConf.CTE_RECURSION_CACHE_MODE)) + + // We store the initial and generated children of the loop in this buffer. + // Please note that all elements are cached because of performance reasons as they are needed for + // next iteration. + @transient private val unionDFs = mutable.ArrayBuffer.empty[DataFrame] + + override def innerChildren: Seq[QueryPlan[_]] = Seq(anchor, recursion) + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + + private def cacheAndCount(plan: LogicalPlan, limit: Option[Long]) = { + val limitedPlan = limit.map(l => Limit(Literal(l.toInt), plan)).getOrElse(plan) + val df = Dataset.ofRows(session, limitedPlan) + val cachedDF = cacheMode match { + case CTERecursionCacheMode.NONE => df + case CTERecursionCacheMode.REPARTITION => df.repartition() + case CTERecursionCacheMode.PERSIST => df.persist() + case CTERecursionCacheMode.LOCAL_CHECKPOINT => df.localCheckpoint() + case CTERecursionCacheMode.CHECKPOINT => df.checkpoint() + } + val count = cachedDF.count() + (cachedDF, count) + } + + override protected def doExecute(): RDD[InternalRow] = { + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + val numOutputRows = longMetric("numOutputRows") + + val unionChildren = mutable.ArrayBuffer.empty[LogicalRDD] + var currentLimit = limit.map(_.toLong) + var (prevDF, prevCount) = cacheAndCount(anchor, currentLimit) + + var currentLevel = 0 + while (prevCount > 0 && currentLimit.forall(_ > 0)) { + unionDFs += prevDF + + if (levelLimit != -1 && currentLevel > levelLimit) { + throw new SparkException(s"Recursion level limit ${levelLimit} reached but query has not " + + s"exhausted, try increasing ${SQLConf.CTE_RECURSION_LEVEL_LIMIT.key}") + } + + // Inherit stats and constraints from the dataset of the previous iteration + val prevPlan = LogicalRDD.fromDataset(prevDF.queryExecution.toRdd, prevDF, prevDF.isStreaming) + .newInstance() + unionChildren += prevPlan + numOutputRows += prevCount + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) + + val newRecursion = recursion.transform { + case r: UnionLoopRef => + val prevRefPlan = if (r.accumulated && unionChildren.length > 1) { + Union(unionChildren.toSeq) + } else { + prevPlan + } + val prevPlanToRefMapping = prevRefPlan.output.zip(r.output).map { + case (fa, ta) => Alias(fa, ta.name)(ta.exprId) + } + Project(prevPlanToRefMapping, prevRefPlan) + } + + val (df, count) = cacheAndCount(newRecursion, currentLimit) + prevDF = df + prevCount = count + + currentLimit = currentLimit.map(_ - count) + currentLevel += 1 + } + + cacheMode match { + case CTERecursionCacheMode.PERSIST => prevDF.unpersist() + case _ => + } + + if (unionChildren.isEmpty) { + new EmptyRDD[InternalRow](sparkContext) + } else if (unionChildren.length == 1) { + Dataset.ofRows(session, unionChildren.head).queryExecution.toRdd + } else { + Dataset.ofRows(session, Union(unionChildren.toSeq)).queryExecution.toRdd + } + } + + override def cleanupResources(): Unit = { + try { + if (unionDFs != null) { + cacheMode match { + case CTERecursionCacheMode.PERSIST => unionDFs.foreach(_.unpersist()) + case _ => + } + } + } finally { + super.cleanupResources() + } + } + + override def doCanonicalize(): SparkPlan = + super.doCanonicalize().asInstanceOf[UnionLoopExec] + .copy(anchor = anchor.canonicalized, recursion = recursion.canonicalized) + + override def verboseStringWithOperatorId(): String = { + s""" + |$formattedNodeName + |Loop id: $loopId + |${QueryPlan.generateFieldString("Output", output)} + |Limit: $limit + |""".stripMargin + } +} + /** * Physical plan for returning a new RDD that has exactly `numPartitions` partitions. * Similar to coalesce defined on an [[RDD]], this operation results in a narrow dependency, e.g. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala index 45fc2a0765c0a..6937c504ff1ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, RowOrdering, SortOrder} -import org.apache.spark.sql.catalyst.plans.physical +import org.apache.spark.sql.catalyst.plans.{physical, QueryPlan} import org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning import org.apache.spark.sql.catalyst.util.{truncatedString, InternalRowComparableWrapper} import org.apache.spark.sql.connector.read.{HasPartitionKey, InputPartition, PartitionReaderFactory, Scan} -import org.apache.spark.sql.execution.{ExplainUtils, LeafExecNode, SQLExecution} +import org.apache.spark.sql.execution.{LeafExecNode, SQLExecution} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.connector.SupportsMetadata @@ -87,7 +87,7 @@ trait DataSourceV2ScanExecBase extends LeafExecNode { } s""" |$formattedNodeName - |${ExplainUtils.generateFieldString("Output", output)} + |${QueryPlan.generateFieldString("Output", output)} |${metaDataStr.mkString("\n")} |""".stripMargin } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala index c02beea4f879c..a87df65b4131d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala @@ -21,6 +21,7 @@ import org.apache.spark.broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder} +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.execution._ @@ -87,7 +88,7 @@ case class ReusedExchangeExec(override val output: Seq[Attribute], child: Exchan val reuse_op_str = ExplainUtils.getOpId(child) s""" |$formattedNodeName [Reuses operator id: $reuse_op_str] - |${ExplainUtils.generateFieldString("Output", output)} + |${QueryPlan.generateFieldString("Output", output)} |""".stripMargin } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BaseJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BaseJoinExec.scala index 50d76aafa2d9d..6760be28ec6c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BaseJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BaseJoinExec.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.{JoinType, QueryPlan} import org.apache.spark.sql.execution.{BinaryExecNode, ExplainUtils} /** @@ -42,16 +42,16 @@ trait BaseJoinExec extends BinaryExecNode { if (leftKeys.nonEmpty || rightKeys.nonEmpty) { s""" |$formattedNodeName - |${ExplainUtils.generateFieldString("Left keys", leftKeys)} - |${ExplainUtils.generateFieldString("Right keys", rightKeys)} - |${ExplainUtils.generateFieldString("Join type", joinType.toString)} - |${ExplainUtils.generateFieldString("Join condition", joinCondStr)} + |${QueryPlan.generateFieldString("Left keys", leftKeys)} + |${QueryPlan.generateFieldString("Right keys", rightKeys)} + |${QueryPlan.generateFieldString("Join type", joinType.toString)} + |${QueryPlan.generateFieldString("Join condition", joinCondStr)} |""".stripMargin } else { s""" |$formattedNodeName - |${ExplainUtils.generateFieldString("Join type", joinType.toString)} - |${ExplainUtils.generateFieldString("Join condition", joinCondStr)} + |${QueryPlan.generateFieldString("Join type", joinType.toString)} + |${QueryPlan.generateFieldString("Join condition", joinCondStr)} |""".stripMargin } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out index c37acb7879c4a..c1ed789a78825 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out @@ -411,14 +411,14 @@ CREATE TEMPORARY VIEW "myview"("c1") AS -- !query analysis CreateViewCommand `myview`, [(c1,None)], WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v", false, false, LocalTempView, true +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias v : +- Project [1#x AS a#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [a#x] +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query @@ -431,14 +431,14 @@ Project [a1#x AS a2#x] +- View (`myview`, [c1#x]) +- Project [cast(a#x as int) AS c1#x] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias v : +- Project [1#x AS a#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [a#x] +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query 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 4aac75ec45a93..2c1ca80f670d0 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 @@ -4,13 +4,13 @@ CREATE TABLE cte_tbl USING csv AS WITH s AS (SELECT 42 AS col) SELECT * FROM s -- !query analysis CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`cte_tbl`, ErrorIfExists, [col] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -26,13 +26,13 @@ CREATE TEMPORARY VIEW cte_view AS WITH s AS (SELECT 42 AS col) SELECT * FROM s -- !query analysis CreateViewCommand `cte_view`, WITH s AS (SELECT 42 AS col) SELECT * FROM s, false, false, LocalTempView, true +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -43,13 +43,13 @@ Project [col#x] +- View (`cte_view`, [col#x]) +- Project [cast(col#x as int) AS col#x] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -58,13 +58,13 @@ INSERT INTO cte_tbl SELECT * FROM S -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/cte_tbl, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/cte_tbl], Append, `spark_catalog`.`default`.`cte_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/cte_tbl), [col] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [43 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias S - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -80,13 +80,13 @@ INSERT INTO cte_tbl WITH s AS (SELECT 44 AS col) SELECT * FROM s -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/cte_tbl, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/cte_tbl], Append, `spark_catalog`.`default`.`cte_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/cte_tbl), [col] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [44 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, 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 f1a302b06f2a8..2528437725e47 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 @@ -7,18 +7,18 @@ WITH t as ( SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -30,14 +30,14 @@ SELECT max(c) FROM ( Aggregate [max(c#x) AS max(c)#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -48,13 +48,13 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [1 AS 1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- OneRowRelation @@ -94,22 +94,22 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [2#x], false +: +- CTERelationRef xxxx, true, [2#x], false, false +- Project [2#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [2#x], false + +- CTERelationRef xxxx, true, [2#x], false, false -- !query @@ -126,29 +126,29 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- Aggregate [max(c#x) AS max(c)#x] : : +- SubqueryAlias __auto_generated_subquery_name : : +- WithCTE -: : :- CTERelationDef xxxx, false +: : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [2#x AS c#x] : : : +- Project [2 AS 2#x] : : : +- OneRowRelation : : +- Project [c#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c#x], false +: : +- CTERelationRef xxxx, true, [c#x], false, false : +- OneRowRelation +- Project [scalarsubquery()#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [scalarsubquery()#x], false + +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false -- !query @@ -165,31 +165,31 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [3 AS 3#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [3#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [3#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [3#x], false +: +- CTERelationRef xxxx, true, [3#x], false, false +- Project [3#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [3#x], false + +- CTERelationRef xxxx, true, [3#x], false, false -- !query @@ -200,12 +200,12 @@ SELECT max(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -214,7 +214,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -227,12 +227,12 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -243,7 +243,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -257,17 +257,17 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [3#x AS c#x] : +- Project [3 AS 3#x] @@ -278,7 +278,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -289,19 +289,19 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [2#x] : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [2#x], false + : +- CTERelationRef xxxx, true, [2#x], false, false +- OneRowRelation @@ -315,20 +315,20 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- WithCTE - : : :- CTERelationDef xxxx, false + : : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [2 AS 2#x] : : : +- OneRowRelation : : +- Project [2#x] : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [2#x], false + : : +- CTERelationRef xxxx, true, [2#x], false, false : +- OneRowRelation +- OneRowRelation @@ -344,25 +344,25 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- WithCTE - : : :- CTERelationDef xxxx, false + : : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [3 AS 3#x] : : : +- OneRowRelation : : +- Project [3#x] : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [3#x], false + : : +- CTERelationRef xxxx, true, [3#x], false, false : +- OneRowRelation +- OneRowRelation @@ -376,7 +376,7 @@ WHERE c IN ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] @@ -384,16 +384,16 @@ WithCTE +- Project [c#x] +- Filter c#x IN (list#x []) : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [2#x AS c#x] : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [c#x] : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [c#x], false + : +- CTERelationRef xxxx, true, [c#x], false, false +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -406,22 +406,22 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -434,22 +434,22 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias abc : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias aBc : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x] : +- SubqueryAlias aBC -: +- CTERelationRef xxxx, true, [2#x], false +: +- CTERelationRef xxxx, true, [2#x], false, false +- Project [2#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [2#x], false + +- CTERelationRef xxxx, true, [2#x], false, false -- !query @@ -460,19 +460,19 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias abc : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias aBc : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [2#x] : +- SubqueryAlias aBC - : +- CTERelationRef xxxx, true, [2#x], false + : +- CTERelationRef xxxx, true, [2#x], false, false +- OneRowRelation @@ -488,23 +488,23 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -519,20 +519,20 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -552,27 +552,27 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, 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 +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, 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 6e55c6fa83cd9..2ccba28b8a016 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 @@ -7,18 +7,18 @@ WITH t as ( SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -30,14 +30,14 @@ SELECT max(c) FROM ( Aggregate [max(c#x) AS max(c)#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -48,13 +48,13 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [1 AS 1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- OneRowRelation @@ -119,29 +119,29 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- Aggregate [max(c#x) AS max(c)#x] : : +- SubqueryAlias __auto_generated_subquery_name : : +- WithCTE -: : :- CTERelationDef xxxx, false +: : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [2#x AS c#x] : : : +- Project [2 AS 2#x] : : : +- OneRowRelation : : +- Project [c#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c#x], false +: : +- CTERelationRef xxxx, true, [c#x], false, false : +- OneRowRelation +- Project [scalarsubquery()#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [scalarsubquery()#x], false + +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false -- !query @@ -177,12 +177,12 @@ SELECT max(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -191,7 +191,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -204,12 +204,12 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -220,7 +220,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -234,17 +234,17 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [3#x AS c#x] : +- Project [3 AS 3#x] @@ -255,7 +255,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -350,22 +350,22 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -420,23 +420,23 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -451,20 +451,20 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -484,27 +484,27 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, 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 +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, 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 new file mode 100644 index 0000000000000..49714ea4dc071 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out @@ -0,0 +1,1064 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +WITH r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`r`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 67, + "stopIndex" : 67, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r AS ( + SELECT 0 AS level + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- UnionLoop xxxx +: :- Project [0 AS level#x] +: : +- OneRowRelation +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- Filter (level#x < 9) +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [level#x], false ++- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- Filter (level#x < 9) +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [level#x], false ++- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +WITH RECURSIVE r(c) AS ( + SELECT 'a' + UNION ALL + SELECT c || char(ascii(substr(c, -1)) + 1) FROM r WHERE LENGTH(c) < 10 +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [a#x AS c#x] +: +- UnionLoop xxxx +: :- Project [a AS a#x] +: : +- OneRowRelation +: +- Project [concat(c#x, char(cast((ascii(substr(c#x, -1, 2147483647)) + 1) as bigint))) AS concat(c, char((ascii(substr(c, -1, 2147483647)) + 1)))#x] +: +- Filter (length(c#x) < 10) +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [c#x], false ++- Project [c#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [c#x], false, false + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [level#x], false ++- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r LIMIT 10 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [level#x], false ++- GlobalLimit 10 + +- LocalLimit 10 + +- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION + SELECT (level + 1) % 10 FROM r +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoop xxxx +: :- Distinct +: : +- LocalRelation [col1#x] +: +- Except false +: :- Project [((level#x + 1) % 10) AS ((level + 1) % 10)#x] +: : +- SubqueryAlias r +: : +- UnionLoopRef xxxx, [level#x], false +: +- UnionLoopRef xxxx, [level#x], true ++- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT level, level FROM r ORDER BY 1 LIMIT 10 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [level#x], false ++- GlobalLimit 10 + +- LocalLimit 10 + +- Sort [level#x ASC NULLS FIRST], true + +- Project [level#x, level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +WITH RECURSIVE r(level) AS ( + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "WITH RECURSIVE r(level) AS (\n SELECT level + 1 FROM r WHERE level < 9\n)\nSELECT * FROM r" + } ] +} + + +-- !query +WITH RECURSIVE r(level) AS ( + SELECT level + 1 FROM r WHERE level < 9 + UNION ALL + VALUES 0 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`level`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 66, + "fragment" : "level" + } ] +} + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + INTERSECT + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 111, + "fragment" : "WITH RECURSIVE r(level) AS (\n VALUES 0\n INTERSECT\n SELECT level + 1 FROM r WHERE level < 9\n)\nSELECT * FROM r" + } ] +} + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE (SELECT SUM(level) FROM r) < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`r`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 109, + "stopIndex" : 109, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE + t1 AS ( + SELECT 1 AS level + UNION ( + WITH t2 AS (SELECT level + 1 FROM t1 WHERE level < 10) + SELECT * FROM t2 + ) + ) +SELECT * FROM t1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 100, + "stopIndex" : 101, + "fragment" : "t1" + } ] +} + + +-- !query +SET spark.sql.legacy.ctePrecedencePolicy=CORRECTED +-- !query analysis +SetCommand (spark.sql.legacy.ctePrecedencePolicy,Some(CORRECTED)) + + +-- !query +WITH + t1 AS (SELECT 1), + t2 AS ( + WITH RECURSIVE + t1 AS ( + SELECT 1 AS level + UNION ( + WITH t3 AS (SELECT level + 1 FROM t1 WHERE level < 10) + SELECT * FROM t3 + ) + ) + SELECT * FROM t1 + ) +SELECT * FROM t2 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 155, + "stopIndex" : 156, + "fragment" : "t1" + } ] +} + + +-- !query +SET spark.sql.legacy.ctePrecedencePolicy=EXCEPTION +-- !query analysis +SetCommand (spark.sql.legacy.ctePrecedencePolicy,Some(EXCEPTION)) + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT r1.level + 1, r1.data + FROM r AS r1 + JOIN r AS r2 ON r2.data = r1.data + WHERE r1.level < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836" +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + LEFT OUTER JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x, col2#x AS data#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x, col2#x] +: +- Project [(level#x + 1) AS (level + 1)#x, data#x] +: +- Filter (level#x < 9) +: +- Join LeftOuter, (data#x = data#x) +: :- SubqueryAlias r +: : +- UnionLoopRef xxxx, [level#x, data#x], false +: +- SubqueryAlias t +: +- Project [0 AS data#x] +: +- OneRowRelation ++- Project [level#x, data#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x, data#x], false, false + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM ( + SELECT 0 AS data + ) AS t + LEFT OUTER JOIN r ON r.data = t.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 148, + "stopIndex" : 148, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM ( + SELECT 0 AS data + ) AS t + RIGHT OUTER JOIN r ON r.data = t.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x, col2#x AS data#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x, col2#x] +: +- Project [(level#x + 1) AS (level + 1)#x, data#x] +: +- Filter (level#x < 9) +: +- Join RightOuter, (data#x = data#x) +: :- SubqueryAlias t +: : +- Project [0 AS data#x] +: : +- OneRowRelation +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [level#x, data#x], false ++- Project [level#x, data#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x, data#x], false, false + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + RIGHT OUTER JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 98, + "stopIndex" : 98, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + FULL OUTER JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 98, + "stopIndex" : 98, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + LEFT SEMI JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x, col2#x AS data#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x, col2#x] +: +- Project [(level#x + 1) AS (level + 1)#x, data#x] +: +- Filter (level#x < 9) +: +- Join LeftSemi, (data#x = data#x) +: :- SubqueryAlias r +: : +- UnionLoopRef xxxx, [level#x, data#x], false +: +- SubqueryAlias t +: +- Project [0 AS data#x] +: +- OneRowRelation ++- Project [level#x, data#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x, data#x], false, false + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, data + FROM ( + SELECT 0 AS level, 0 AS data + ) AS t + LEFT SEMI JOIN r ON r.data = t.data AND r.level < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 157, + "stopIndex" : 157, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + LEFT ANTI JOIN ( + SELECT -1 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x, col2#x AS data#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x, col2#x] +: +- Project [(level#x + 1) AS (level + 1)#x, data#x] +: +- Filter (level#x < 9) +: +- Join LeftAnti, (data#x = data#x) +: :- SubqueryAlias r +: : +- UnionLoopRef xxxx, [level#x, data#x], false +: +- SubqueryAlias t +: +- Project [-1 AS data#x] +: +- OneRowRelation ++- Project [level#x, data#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x, data#x], false, false + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, data + FROM ( + SELECT 0 AS level, 0 AS data + ) AS t + LEFT ANTI JOIN r ON r.data = t.data AND r.level < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 157, + "stopIndex" : 157, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 1L) + UNION ALL + SELECT MAX(level) + 1, SUM(data) FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 105, + "stopIndex" : 105, + "fragment" : "r" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW rv AS +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query analysis +CreateViewCommand `rv`, WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r, false, false, LocalTempView, true + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias r + : +- Project [col1#x AS level#x] + : +- Union false, false + : :- LocalRelation [col1#x] + : +- Project [(level#x + 1) AS (level + 1)#x] + : +- Filter (level#x < 9) + : +- SubqueryAlias r + : +- CTERelationRef xxxx, true, [level#x], false, true + +- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +SELECT * FROM rv +-- !query analysis +Project [level#x] ++- SubqueryAlias rv + +- View (`rv`, [level#x]) + +- Project [cast(level#x as int) AS level#x] + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias r + : +- Project [col1#x AS level#x] + : +- UnionLoop xxxx + : :- LocalRelation [col1#x] + : +- Project [(level#x + 1) AS (level + 1)#x] + : +- Filter (level#x < 9) + : +- SubqueryAlias r + : +- UnionLoopRef xxxx, [level#x], false + +- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +DROP VIEW rv +-- !query analysis +DropTempViewCommand rv + + +-- !query +CREATE TABLE rt(level INT) USING csv +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`rt`, false + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +INSERT INTO rt SELECT * FROM r +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/rt, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/rt], Append, `spark_catalog`.`default`.`rt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/rt), [level] ++- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias r + : +- Project [col1#x AS level#x] + : +- UnionLoop xxxx + : :- LocalRelation [col1#x] + : +- Project [(level#x + 1) AS (level + 1)#x] + : +- Filter (level#x < 9) + : +- SubqueryAlias r + : +- UnionLoopRef xxxx, [level#x], false + +- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +SELECT * from rt +-- !query analysis +Project [level#x] ++- SubqueryAlias spark_catalog.default.rt + +- Relation spark_catalog.default.rt[level#x] csv + + +-- !query +DROP TABLE rt +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.rt + + +-- !query +CREATE TABLE rt2(level INT) USING csv +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`rt2`, false + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +FROM r +INSERT INTO rt2 SELECT * +INSERT INTO rt2 SELECT * +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "RECURSIVE_CTE_WHEN_INLINING_IS_FORCED", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 160, + "fragment" : "WITH RECURSIVE r(level) AS (\n VALUES (0)\n UNION ALL\n SELECT level + 1 FROM r WHERE level < 9\n)\nFROM r\nINSERT INTO rt2 SELECT *\nINSERT INTO rt2 SELECT *" + } ] +} + + +-- !query +DROP TABLE rt2 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.rt2 + + +-- !query +WITH RECURSIVE + r1 AS ( + SELECT 0 AS level + UNION ALL + SELECT level + 1 FROM r1 WHERE level < 9 + ), + r2 AS ( + SELECT 10 AS level + UNION ALL + SELECT level + 1 FROM r2 WHERE level < 19 + ) +SELECT * +FROM r1 +JOIN r2 ON r2.level = r1.level + 10 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r1 +: +- UnionLoop xxxx +: :- Project [0 AS level#x] +: : +- OneRowRelation +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- Filter (level#x < 9) +: +- SubqueryAlias r1 +: +- UnionLoopRef xxxx, [level#x], false +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r2 +: +- UnionLoop xxxx +: :- Project [10 AS level#x] +: : +- OneRowRelation +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- Filter (level#x < 19) +: +- SubqueryAlias r2 +: +- UnionLoopRef xxxx, [level#x], false ++- Project [level#x, level#x] + +- Join Inner, (level#x = (level#x + 10)) + :- SubqueryAlias r1 + : +- CTERelationRef xxxx, true, [level#x], false, false + +- SubqueryAlias r2 + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +WITH RECURSIVE r AS ( + SELECT 0 AS level + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * +FROM r AS r1 +JOIN r AS r2 ON r2.level = r1.level +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- UnionLoop xxxx +: :- Project [0 AS level#x] +: : +- OneRowRelation +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- Filter (level#x < 9) +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [level#x], false ++- Project [level#x, level#x] + +- Join Inner, (level#x = level#x) + :- SubqueryAlias r1 + : +- SubqueryAlias r + : +- CTERelationRef xxxx, true, [level#x], false, false + +- SubqueryAlias r2 + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +WITH RECURSIVE r2 AS ( + WITH RECURSIVE r1 AS ( + SELECT 0 AS innerlevel + UNION ALL + SELECT innerlevel + 1 FROM r1 WHERE innerlevel < 3 + ) + SELECT 0 AS outerlevel, innerlevel FROM r1 + UNION ALL + SELECT outerlevel + 1, innerlevel FROM r2 WHERE outerlevel < 3 +) +SELECT * FROM r2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r1 +: +- UnionLoop xxxx +: :- Project [0 AS innerlevel#x] +: : +- OneRowRelation +: +- Project [(innerlevel#x + 1) AS (innerlevel + 1)#x] +: +- Filter (innerlevel#x < 3) +: +- SubqueryAlias r1 +: +- UnionLoopRef xxxx, [innerlevel#x], false +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r2 +: +- UnionLoop xxxx +: :- Project [0 AS outerlevel#x, innerlevel#x] +: : +- SubqueryAlias r1 +: : +- CTERelationRef xxxx, true, [innerlevel#x], 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 + + +-- !query +WITH RECURSIVE r(level) AS ( + WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 + ) + SELECT * FROM r + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- Filter (level#x < 3) +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [level#x], false +:- CTERelationDef xxxx, false, false +: +- SubqueryAlias r +: +- Project [level#x AS level#x] +: +- Union false, false +: :- Project [level#x] +: : +- SubqueryAlias r +: : +- CTERelationRef xxxx, true, [level#x], false, false +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- Filter (level#x < 3) +: +- SubqueryAlias r +: +- CTERelationRef xxxx, true, [level#x], false, false ++- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +WITH RECURSIVE r(level) AS ( + (WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 + ) + SELECT * FROM r) + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 +) +SELECT * FROM r +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- Filter (level#x < 3) +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [level#x], false +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias r +: +- Project [level#x AS level#x] +: +- UnionLoop xxxx +: :- Project [level#x] +: : +- SubqueryAlias r +: : +- CTERelationRef xxxx, true, [level#x], false, false +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- Filter (level#x < 3) +: +- SubqueryAlias r +: +- UnionLoopRef xxxx, [level#x], false ++- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +CREATE TEMPORARY VIEW routes(origin, destination) AS VALUES + ('New York', 'Washington'), + ('New York', 'Boston'), + ('Boston', 'New York'), + ('Washington', 'Boston'), + ('Washington', 'Raleigh') +-- !query analysis +CreateViewCommand `routes`, [(origin,None), (destination,None)], VALUES + ('New York', 'Washington'), + ('New York', 'Boston'), + ('Boston', 'New York'), + ('Washington', 'Boston'), + ('Washington', 'Raleigh'), false, false, LocalTempView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +WITH RECURSIVE destinations_from_new_york AS ( + SELECT 'New York' AS destination, ARRAY('New York') AS path, 0 AS length + UNION ALL + SELECT r.destination, CONCAT(d.path, ARRAY(r.destination)), d.length + 1 + FROM routes AS r + JOIN destinations_from_new_york AS d ON d.destination = r.origin AND NOT ARRAY_CONTAINS(d.path, r.destination) +) +SELECT * FROM destinations_from_new_york +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias destinations_from_new_york +: +- UnionLoop xxxx +: :- Project [New York AS destination#x, array(New York) AS path#x, 0 AS length#x] +: : +- OneRowRelation +: +- Project [destination#x, concat(path#x, array(destination#x)) AS concat(path, array(destination))#x, (length#x + 1) AS (length + 1)#x] +: +- Join Inner, ((destination#x = origin#x) AND NOT array_contains(path#x, destination#x)) +: :- SubqueryAlias r +: : +- SubqueryAlias routes +: : +- View (`routes`, [origin#x,destination#x]) +: : +- Project [cast(col1#x as string) AS origin#x, cast(col2#x as string) AS destination#x] +: : +- LocalRelation [col1#x, col2#x] +: +- SubqueryAlias d +: +- SubqueryAlias destinations_from_new_york +: +- 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 + + +-- !query +DROP VIEW routes +-- !query analysis +DropTempViewCommand routes + + +-- !query +WITH RECURSIVE fibonacci AS ( + VALUES (0, 1) AS t(a, b) + UNION ALL + SELECT b, a + b FROM fibonacci WHERE a < 10 +) +SELECT a FROM fibonacci ORDER BY a +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias fibonacci +: +- UnionLoop xxxx +: :- SubqueryAlias t +: : +- LocalRelation [a#x, b#x] +: +- Project [b#x AS b#x, (a + b)#x AS (a + b)#x] +: +- Project [b#x, (a#x + b#x) AS (a + b)#x] +: +- Filter (a#x < 10) +: +- SubqueryAlias fibonacci +: +- UnionLoopRef xxxx, [a#x, b#x], false ++- Sort [a#x ASC NULLS FIRST], true + +- Project [a#x] + +- SubqueryAlias fibonacci + +- CTERelationRef xxxx, true, [a#x, b#x], 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 1d9d812875c44..d3b389f3648e8 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 @@ -62,7 +62,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException WITH t AS (SELECT 1 FROM t) SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- SubqueryAlias t @@ -73,7 +73,7 @@ WithCTE : +- LocalRelation [id#x] +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -100,7 +100,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [id#x] : +- SubqueryAlias t2 @@ -109,17 +109,17 @@ WithCTE : +- Project [id#x] : +- SubqueryAlias t : +- LocalRelation [id#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [id#x], false +: +- CTERelationRef xxxx, true, [id#x], false, false +- Project [id#x, 2#x] +- Join Cross :- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [id#x], false + : +- CTERelationRef xxxx, true, [id#x], false, false +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [2#x], false + +- CTERelationRef xxxx, true, [2#x], false, false -- !query @@ -134,7 +134,7 @@ FROM CTE1 t1 CROSS JOIN CTE1 t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias CTE1 : +- Project [id#x AS id#x] : +- Join Cross @@ -157,10 +157,10 @@ WithCTE +- Join Cross :- SubqueryAlias t1 : +- SubqueryAlias CTE1 - : +- CTERelationRef xxxx, true, [id#x], false + : +- CTERelationRef xxxx, true, [id#x], false, false +- SubqueryAlias t2 +- SubqueryAlias CTE1 - +- CTERelationRef xxxx, true, [id#x], false + +- CTERelationRef xxxx, true, [id#x], false, false -- !query @@ -168,7 +168,7 @@ WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE x = 1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS x#x] : +- Project [1 AS 1#x] @@ -176,7 +176,7 @@ WithCTE +- Project [x#x] +- Filter (x#x = 1) +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -184,7 +184,7 @@ WITH t(x, y) AS (SELECT 1, 2) SELECT * FROM t WHERE x = 1 AND y = 2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS x#x, 2#x AS y#x] : +- Project [1 AS 1#x, 2 AS 2#x] @@ -192,7 +192,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 + +- CTERelationRef xxxx, true, [x#x, y#x], false, false -- !query @@ -200,14 +200,14 @@ WITH t(x, x) AS (SELECT 1, 2) SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS x#x, 2#x AS x#x] : +- Project [1 AS 1#x, 2 AS 2#x] : +- OneRowRelation +- Project [x#x, x#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x, x#x], false + +- CTERelationRef xxxx, true, [x#x, x#x], false, false -- !query @@ -299,56 +299,56 @@ WITH w1(c1) AS SELECT * FROM w1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w8 : +- Project [1#x AS c8#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w7 : +- Project [c8#x AS c7#x] : +- Project [c8#x] : +- SubqueryAlias w8 -: +- CTERelationRef xxxx, true, [c8#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c8#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w6 : +- Project [c7#x AS c6#x] : +- Project [c7#x] : +- SubqueryAlias w7 -: +- CTERelationRef xxxx, true, [c7#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c7#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w5 : +- Project [c6#x AS c5#x] : +- Project [c6#x] : +- SubqueryAlias w6 -: +- CTERelationRef xxxx, true, [c6#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c6#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w4 : +- Project [c5#x AS c4#x] : +- Project [c5#x] : +- SubqueryAlias w5 -: +- CTERelationRef xxxx, true, [c5#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c5#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w3 : +- Project [c4#x AS c3#x] : +- Project [c4#x] : +- SubqueryAlias w4 -: +- CTERelationRef xxxx, true, [c4#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c4#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w2 : +- Project [c3#x AS c2#x] : +- Project [c3#x] : +- SubqueryAlias w3 -: +- CTERelationRef xxxx, true, [c3#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c3#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w1 : +- Project [c2#x AS c1#x] : +- Project [c2#x] : +- SubqueryAlias w2 -: +- CTERelationRef xxxx, true, [c2#x], false +: +- CTERelationRef xxxx, true, [c2#x], false, false +- Project [c1#x] +- SubqueryAlias w1 - +- CTERelationRef xxxx, true, [c1#x], false + +- CTERelationRef xxxx, true, [c1#x], false, false -- !query @@ -378,14 +378,14 @@ WITH same_name AS (SELECT 42) SELECT * FROM same_name, (SELECT 10) AS same_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias same_name : +- Project [42 AS 42#x] : +- OneRowRelation +- Project [42#x, 10#x] +- Join Inner :- SubqueryAlias same_name - : +- CTERelationRef xxxx, true, [42#x], false + : +- CTERelationRef xxxx, true, [42#x], false, false +- SubqueryAlias same_name +- Project [10 AS 10#x] +- OneRowRelation @@ -418,13 +418,13 @@ WITH q AS (SELECT 'foo' AS x) SELECT x, typeof(x) FROM q -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q : +- Project [foo AS x#x] : +- OneRowRelation +- Project [x#x, typeof(x#x) AS typeof(x)#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -478,13 +478,13 @@ SELECT * FROM Project [y#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias q : +- Project [1 AS x#x] : +- OneRowRelation +- Project [(x#x + 1) AS y#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -492,13 +492,13 @@ select (with q as (select 1 x) select * from q) -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [1 AS x#x] : : +- OneRowRelation : +- Project [x#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- OneRowRelation @@ -507,13 +507,13 @@ select 1 in (with q as (select 1) select * from q) -- !query analysis Project [1 IN (list#x []) AS (1 IN (listquery()))#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [1 AS 1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- OneRowRelation @@ -548,11 +548,11 @@ from T1 z -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T1 : +- Project [1 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T1 : +- Project [2 AS b#x] : +- OneRowRelation @@ -561,14 +561,14 @@ WithCTE :- Join Inner : :- SubqueryAlias x : : +- SubqueryAlias T1 - : : +- CTERelationRef xxxx, true, [a#x], false + : : +- CTERelationRef xxxx, true, [a#x], false, false : +- SubqueryAlias y : +- Project [b#x] : +- SubqueryAlias T1 - : +- CTERelationRef xxxx, true, [b#x], false + : +- CTERelationRef xxxx, true, [b#x], false, false +- SubqueryAlias z +- SubqueryAlias T1 - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query @@ -579,15 +579,15 @@ from (WITH TtTt as (select 3 c) select * from ttTT, `tttT_2`) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias TTtt : +- Project [1 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias tTTt_2 : +- Project [2 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias TtTt : +- Project [3 AS c#x] : +- OneRowRelation @@ -596,9 +596,9 @@ WithCTE +- Project [c#x, a#x] +- Join Inner :- SubqueryAlias ttTT - : +- CTERelationRef xxxx, true, [c#x], false + : +- CTERelationRef xxxx, true, [c#x], false, false +- SubqueryAlias tttT_2 - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query @@ -608,13 +608,13 @@ from (select 1 x, 2 y) T -- !query analysis Project [scalar-subquery#x [x#x] AS scalarsubquery(x)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [outer(x#x)] : : +- OneRowRelation : +- Project [x#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- SubqueryAlias T +- Project [1 AS x#x, 2 AS y#x] +- OneRowRelation @@ -627,13 +627,13 @@ from (select 1 x, 2 y) T -- !query analysis Project [scalar-subquery#x [x#x && y#x] AS scalarsubquery(x, y)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [3 AS z#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 +: +- CTERelationRef xxxx, true, [z#x], false, false +- SubqueryAlias T +- Project [1 AS x#x, 2 AS y#x] +- OneRowRelation @@ -645,20 +645,20 @@ select * from (with q2 as (select * from q1) select * from q2) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [1 AS x#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q2 : +- Project [x#x] : +- SubqueryAlias q1 -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- Project [x#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [x#x] +- SubqueryAlias q2 - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -667,20 +667,20 @@ select * from (with q1 as (select x+1 from q1) select * from q1) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [1 AS x#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [(x#x + 1) AS (x + 1)#x] : +- SubqueryAlias q1 -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- Project [(x + 1)#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [(x + 1)#x] +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [(x + 1)#x], false + +- CTERelationRef xxxx, true, [(x + 1)#x], false, false -- !query @@ -707,7 +707,7 @@ with cte1 as (Select id as j from t) select * from cte1 where j = (select max(j) from cte1 as cte2) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [id#x AS j#x] : +- SubqueryAlias t @@ -721,9 +721,9 @@ WithCTE : +- Aggregate [max(j#x) AS max(j)#x] : +- SubqueryAlias cte2 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [j#x], false + : +- CTERelationRef xxxx, true, [j#x], false, false +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [j#x], false + +- CTERelationRef xxxx, true, [j#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out index 26f40c1011140..a42a57a56707c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out @@ -157,6 +157,18 @@ EXPLAIN FORMATTED ExplainCommand CTE [cte1], FormattedMode +-- !query +EXPLAIN FORMATTED + WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 + ) +SELECT * FROM r +-- !query analysis +ExplainCommand CTE [r], FormattedMode + + -- !query EXPLAIN FORMATTED CREATE VIEW explain_view AS diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out index 26f40c1011140..a42a57a56707c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out @@ -157,6 +157,18 @@ EXPLAIN FORMATTED ExplainCommand CTE [cte1], FormattedMode +-- !query +EXPLAIN FORMATTED + WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 + ) +SELECT * FROM r +-- !query analysis +ExplainCommand CTE [r], FormattedMode + + -- !query EXPLAIN FORMATTED CREATE VIEW explain_view AS 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 faa582035ef3a..be7c39f8b1514 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 @@ -1358,14 +1358,14 @@ WITH cte1 AS ( SELECT * FROM cte2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [c1#x] : +- SubqueryAlias spark_catalog.default.t1 : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : +- LocalRelation [col1#x, col2#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte2 : +- Project [c1#x, c2#x] : +- LateralJoin lateral-subquery#x [c1#x], Inner @@ -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 +: +- CTERelationRef xxxx, true, [c1#x], false, false +- Project [c1#x, c2#x] +- SubqueryAlias cte2 - +- CTERelationRef xxxx, true, [c1#x, c2#x], false + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, 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 b80bed6f7c2aa..c3f76ca30e0bf 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 @@ -36,7 +36,7 @@ WITH tmp AS ( SELECT id FROM range(3) WHERE id > (SELECT max(id) FROM tmp) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias tmp : +- Intersect false : :- Project [id#xL] @@ -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 + : +- CTERelationRef xxxx, true, [id#xL], false, false +- Range (0, 3, step=1, splits=None) 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 78a3c60d20dd8..ee1cfe07a9493 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 @@ -89,7 +89,7 @@ FROM cte WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [id#xL AS x#xL] : +- Project [id#xL] @@ -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 + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query @@ -111,7 +111,7 @@ FROM cte WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [id#xL AS x#xL] : +- Project [id#xL] @@ -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 + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query @@ -134,7 +134,7 @@ FROM cte WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [1#xL AS x#xL] : +- Union false, false @@ -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 + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query @@ -167,7 +167,7 @@ FROM cte WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [1#xL AS x#xL] : +- Union false, false @@ -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 + +- CTERelationRef xxxx, true, [x#xL], false, 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 f29b12d5af171..b3cbe733e03b0 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 @@ -4,7 +4,7 @@ WITH q1(x,y) AS (SELECT 1,2) SELECT * FROM q1, q1 AS q2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [1#x AS x#x, 2#x AS y#x] : +- Project [1 AS 1#x, 2 AS 2#x] @@ -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 + : +- CTERelationRef xxxx, true, [x#x, y#x], false, false +- SubqueryAlias q2 +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [x#x, y#x], false + +- CTERelationRef xxxx, true, [x#x, y#x], false, false -- !query @@ -29,6 +29,297 @@ SELECT count(*) FROM ( [Analyzer test output redacted due to nondeterminism] +-- !query +WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t WHERE n < 100 +) +SELECT sum(n) FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [col1#x AS n#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(n#x + 1) AS (n + 1)#x] +: +- Filter (n#x < 100) +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [n#x], false ++- Aggregate [sum(n#x) AS sum(n)#xL] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT (VALUES(1)) +UNION ALL + SELECT n+1 FROM t WHERE n < 5 +) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [scalarsubquery()#x AS n#x] +: +- UnionLoop xxxx +: :- Project [scalar-subquery#x [] AS scalarsubquery()#x] +: : : +- LocalRelation [col1#x] +: : +- OneRowRelation +: +- Project [(n#x + 1) AS (n + 1)#x] +: +- Filter (n#x < 5) +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [n#x], false ++- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +CREATE TEMPORARY VIEW nums AS +WITH RECURSIVE nums (n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM nums WHERE n < 5 +) +SELECT * FROM nums +-- !query analysis +CreateViewCommand `nums`, WITH RECURSIVE nums (n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM nums WHERE n < 5 +) +SELECT * FROM nums, false, false, LocalTempView, true + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias nums + : +- Project [col1#x AS n#x] + : +- Union false, false + : :- LocalRelation [col1#x] + : +- Project [(n#x + 1) AS (n + 1)#x] + : +- Filter (n#x < 5) + : +- SubqueryAlias nums + : +- CTERelationRef xxxx, true, [n#x], false, true + +- Project [n#x] + +- SubqueryAlias nums + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +SELECT * FROM nums +-- !query analysis +Project [n#x] ++- SubqueryAlias nums + +- View (`nums`, [n#x]) + +- Project [cast(n#x as int) AS n#x] + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias nums + : +- Project [col1#x AS n#x] + : +- UnionLoop xxxx + : :- LocalRelation [col1#x] + : +- Project [(n#x + 1) AS (n + 1)#x] + : +- Filter (n#x < 5) + : +- SubqueryAlias nums + : +- UnionLoopRef xxxx, [n#x], false + +- Project [n#x] + +- SubqueryAlias nums + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW nums AS +WITH RECURSIVE nums (n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM nums WHERE n < 6 +) +SELECT * FROM nums +-- !query analysis +CreateViewCommand `nums`, WITH RECURSIVE nums (n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM nums WHERE n < 6 +) +SELECT * FROM nums, false, true, LocalTempView, true + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias nums + : +- Project [col1#x AS n#x] + : +- Union false, false + : :- LocalRelation [col1#x] + : +- Project [(n#x + 1) AS (n + 1)#x] + : +- Filter (n#x < 6) + : +- SubqueryAlias nums + : +- CTERelationRef xxxx, true, [n#x], false, true + +- Project [n#x] + +- SubqueryAlias nums + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +SELECT * FROM nums +-- !query analysis +Project [n#x] ++- SubqueryAlias nums + +- View (`nums`, [n#x]) + +- Project [cast(n#x as int) AS n#x] + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias nums + : +- Project [col1#x AS n#x] + : +- UnionLoop xxxx + : :- LocalRelation [col1#x] + : +- Project [(n#x + 1) AS (n + 1)#x] + : +- Filter (n#x < 6) + : +- SubqueryAlias nums + : +- UnionLoopRef xxxx, [n#x], false + +- Project [n#x] + +- SubqueryAlias nums + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT 1 +UNION + SELECT 10-n FROM t) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [1#x AS n#x] +: +- UnionLoop xxxx +: :- Distinct +: : +- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Except false +: :- Project [(10 - n#x) AS (10 - n)#x] +: : +- SubqueryAlias t +: : +- UnionLoopRef xxxx, [n#x], false +: +- UnionLoopRef xxxx, [n#x], true ++- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t) +SELECT * FROM t LIMIT 10 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [col1#x AS n#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(n#x + 1) AS (n + 1)#x] +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [n#x], false ++- GlobalLimit 10 + +- LocalLimit 10 + +- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT 1 +UNION + SELECT n+1 FROM t) +SELECT * FROM t LIMIT 10 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [1#x AS n#x] +: +- UnionLoop xxxx +: :- Distinct +: : +- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Except false +: :- Project [(n#x + 1) AS (n + 1)#x] +: : +- SubqueryAlias t +: : +- UnionLoopRef xxxx, [n#x], false +: +- UnionLoopRef xxxx, [n#x], true ++- GlobalLimit 10 + +- LocalLimit 10 + +- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +WITH q AS (SELECT 'foo' AS x) +SELECT x FROM q +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, false +: +- SubqueryAlias q +: +- Project [foo AS x#x] +: +- OneRowRelation ++- Project [x#x] + +- SubqueryAlias q + +- CTERelationRef xxxx, true, [x#x], false, false + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT 'foo' +UNION ALL + SELECT n || ' bar' FROM t WHERE length(n) < 20 +) +SELECT n AS is_text FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [foo#x AS n#x] +: +- UnionLoop xxxx +: :- Project [foo AS foo#x] +: : +- OneRowRelation +: +- Project [concat(n#x, bar) AS concat(n, bar)#x] +: +- Filter (length(n#x) < 20) +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [n#x], false ++- Project [n#x AS is_text#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT '7' +UNION ALL + SELECT n+1 FROM t WHERE n < 10 +) +SELECT n FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.DATA_TYPE", + "sqlState" : "42836", + "messageParameters" : { + "fromDataType" : "\"STRING\"", + "toDataType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 71, + "fragment" : "t" + } ] +} + + -- !query CREATE TABLE department ( id INTEGER, -- department ID @@ -103,6 +394,374 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d +- LocalRelation [col1#x, col2#x, col3#x] +-- !query +WITH RECURSIVE subdepartment AS +( + SELECT name as root_name, * FROM department WHERE name = 'A' + + UNION ALL + + SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id +) +SELECT * FROM subdepartment ORDER BY name +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias subdepartment +: +- UnionLoop xxxx +: :- Project [name#x AS root_name#x, id#x, parent_department#x, name#x] +: : +- Filter (name#x = A) +: : +- SubqueryAlias spark_catalog.default.department +: : +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet +: +- Project [root_name#x AS root_name#x, id#x AS id#x, parent_department#x AS parent_department#x, name#x AS name#x] +: +- Project [root_name#x, id#x, parent_department#x, name#x] +: +- Filter (parent_department#x = id#x) +: +- Join Inner +: :- SubqueryAlias d +: : +- SubqueryAlias spark_catalog.default.department +: : +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet +: +- SubqueryAlias sd +: +- SubqueryAlias subdepartment +: +- UnionLoopRef xxxx, [root_name#x, id#x, parent_department#x, name#x], false ++- 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 + + +-- !query +WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +( + SELECT 1, * FROM department WHERE name = 'A' + + UNION ALL + + SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id +) +SELECT * FROM subdepartment ORDER BY name +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias subdepartment +: +- Project [1#x AS level#x, id#x AS id#x, parent_department#x AS parent_department#x, name#x AS name#x] +: +- UnionLoop xxxx +: :- Project [1 AS 1#x, id#x, parent_department#x, name#x] +: : +- Filter (name#x = A) +: : +- SubqueryAlias spark_catalog.default.department +: : +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet +: +- Project [(level#x + 1) AS (level + 1)#x, id#x, parent_department#x, name#x] +: +- Filter (parent_department#x = id#x) +: +- Join Inner +: :- SubqueryAlias d +: : +- SubqueryAlias spark_catalog.default.department +: : +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet +: +- SubqueryAlias sd +: +- SubqueryAlias subdepartment +: +- UnionLoopRef xxxx, [level#x, id#x, parent_department#x, name#x], false ++- 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 + + +-- !query +WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +( + SELECT 1, * FROM department WHERE name = 'A' + + UNION ALL + + SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id +) +SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias subdepartment +: +- Project [1#x AS level#x, id#x AS id#x, parent_department#x AS parent_department#x, name#x AS name#x] +: +- UnionLoop xxxx +: :- Project [1 AS 1#x, id#x, parent_department#x, name#x] +: : +- Filter (name#x = A) +: : +- SubqueryAlias spark_catalog.default.department +: : +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet +: +- Project [(level#x + 1) AS (level + 1)#x, id#x, parent_department#x, name#x] +: +- Filter (parent_department#x = id#x) +: +- Join Inner +: :- SubqueryAlias d +: : +- SubqueryAlias spark_catalog.default.department +: : +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet +: +- SubqueryAlias sd +: +- SubqueryAlias subdepartment +: +- UnionLoopRef xxxx, [level#x, id#x, parent_department#x, name#x], false ++- Sort [name#x ASC NULLS FIRST], true + +- 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 + + +-- !query +WITH RECURSIVE subdepartment AS +( + SELECT * FROM department WHERE name = 'A' +) +SELECT * FROM subdepartment ORDER BY name +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, false +: +- SubqueryAlias subdepartment +: +- Project [id#x, parent_department#x, name#x] +: +- Filter (name#x = A) +: +- SubqueryAlias spark_catalog.default.department +: +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet ++- 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 + + +-- !query +SET spark.sql.cteRecursionLevelLimit=200 +-- !query analysis +SetCommand (spark.sql.cteRecursionLevelLimit,Some(200)) + + +-- !query +SELECT count(*) FROM ( + WITH RECURSIVE t(n) AS ( + SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 200 + ) + SELECT * FROM t) AS t WHERE n < ( + SELECT count(*) FROM ( + WITH RECURSIVE t(n) AS ( + SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 100 + ) + SELECT * FROM t WHERE n < 50000 + ) AS t WHERE n < 100) +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (cast(n#x as bigint) < scalar-subquery#x []) + : +- Aggregate [count(1) AS count(1)#xL] + : +- Filter (n#x < 100) + : +- SubqueryAlias t + : +- WithCTE + : :- CTERelationDef xxxx, false, true + : : +- SubqueryAlias t + : : +- Project [1#x AS n#x] + : : +- UnionLoop xxxx + : : :- Project [1 AS 1#x] + : : : +- OneRowRelation + : : +- Project [(n#x + 1) AS (n + 1)#x] + : : +- Filter (n#x < 100) + : : +- SubqueryAlias t + : : +- UnionLoopRef xxxx, [n#x], false + : +- Project [n#x] + : +- Filter (n#x < 50000) + : +- SubqueryAlias t + : +- CTERelationRef xxxx, true, [n#x], false, false + +- SubqueryAlias t + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias t + : +- Project [1#x AS n#x] + : +- UnionLoop xxxx + : :- Project [1 AS 1#x] + : : +- OneRowRelation + : +- Project [(n#x + 1) AS (n + 1)#x] + : +- Filter (n#x < 200) + : +- SubqueryAlias t + : +- UnionLoopRef xxxx, [n#x], false + +- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +SET spark.sql.cteRecursionLevelLimit=100 +-- !query analysis +SetCommand (spark.sql.cteRecursionLevelLimit,Some(100)) + + +-- !query +WITH q1(x,y) AS ( + SELECT hundred, sum(ten) FROM tenk1 GROUP BY hundred + ) +SELECT count(*) FROM q1 WHERE y > (SELECT sum(y)/100 FROM q1 qsub) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, false +: +- SubqueryAlias q1 +: +- Project [hundred#x AS x#x, sum(ten)#xL AS y#xL] +: +- Aggregate [hundred#x], [hundred#x, sum(ten#x) AS sum(ten)#xL] +: +- SubqueryAlias spark_catalog.default.tenk1 +: +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet ++- Aggregate [count(1) AS count(1)#xL] + +- Filter (cast(y#xL as double) > scalar-subquery#x []) + : +- 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 + +- SubqueryAlias q1 + +- CTERelationRef xxxx, true, [x#x, y#xL], false, false + + +-- !query +CREATE TEMPORARY VIEW vsubdepartment AS + WITH RECURSIVE subdepartment AS + ( + SELECT * FROM department WHERE name = 'A' + UNION ALL + SELECT d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id + ) + SELECT * FROM subdepartment +-- !query analysis +CreateViewCommand `vsubdepartment`, WITH RECURSIVE subdepartment AS + ( + SELECT * FROM department WHERE name = 'A' + UNION ALL + SELECT d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id + ) + SELECT * FROM subdepartment, false, false, LocalTempView, true + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias subdepartment + : +- Union false, false + : :- Project [id#x, parent_department#x, name#x] + : : +- Filter (name#x = A) + : : +- SubqueryAlias spark_catalog.default.department + : : +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet + : +- Project [id#x, parent_department#x, name#x] + : +- Filter (parent_department#x = id#x) + : +- Join Inner + : :- SubqueryAlias d + : : +- SubqueryAlias spark_catalog.default.department + : : +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet + : +- SubqueryAlias sd + : +- SubqueryAlias subdepartment + : +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, true + +- Project [id#x, parent_department#x, name#x] + +- SubqueryAlias subdepartment + +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false + + +-- !query +SELECT * FROM vsubdepartment ORDER BY name +-- !query analysis +Sort [name#x ASC NULLS FIRST], true ++- Project [id#x, parent_department#x, name#x] + +- SubqueryAlias vsubdepartment + +- View (`vsubdepartment`, [id#x,parent_department#x,name#x]) + +- Project [cast(id#x as int) AS id#x, cast(parent_department#x as int) AS parent_department#x, cast(name#x as string) AS name#x] + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias subdepartment + : +- UnionLoop xxxx + : :- Project [id#x, parent_department#x, name#x] + : : +- Filter (name#x = A) + : : +- SubqueryAlias spark_catalog.default.department + : : +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet + : +- Project [id#x, parent_department#x, name#x] + : +- Filter (parent_department#x = id#x) + : +- Join Inner + : :- SubqueryAlias d + : : +- SubqueryAlias spark_catalog.default.department + : : +- Relation spark_catalog.default.department[id#x,parent_department#x,name#x] parquet + : +- SubqueryAlias sd + : +- SubqueryAlias subdepartment + : +- 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 + + +-- !query +CREATE VIEW sums_1_100 AS +WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t WHERE n < 100 +) +SELECT sum(n) AS sum FROM t +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`sums_1_100`, WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t WHERE n < 100 +) +SELECT sum(n) AS sum FROM t, false, false, PersistedView, true + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias t + : +- Project [col1#x AS n#x] + : +- Union false, false + : :- LocalRelation [col1#x] + : +- Project [(n#x + 1) AS (n + 1)#x] + : +- Filter (n#x < 100) + : +- SubqueryAlias t + : +- CTERelationRef xxxx, true, [n#x], false, true + +- Aggregate [sum(n#x) AS sum#xL] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +SELECT * FROM sums_1_100 +-- !query analysis +Project [sum#xL] ++- SubqueryAlias spark_catalog.default.sums_1_100 + +- View (`spark_catalog`.`default`.`sums_1_100`, [sum#xL]) + +- Project [cast(sum#xL as bigint) AS sum#xL] + +- WithCTE + :- CTERelationDef xxxx, false, true + : +- SubqueryAlias t + : +- Project [col1#x AS n#x] + : +- UnionLoop xxxx + : :- LocalRelation [col1#x] + : +- Project [(n#x + 1) AS (n + 1)#x] + : +- Filter (n#x < 100) + : +- SubqueryAlias t + : +- UnionLoopRef xxxx, [n#x], false + +- Aggregate [sum(n#x) AS sum#xL] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +WITH RECURSIVE t(i,j) AS ( + VALUES (1,2) + UNION ALL + SELECT t2.i, t.j+1 FROM + (SELECT 2 AS i UNION ALL SELECT 3 AS i) AS t2 + JOIN t ON (t2.i = t.i+1)) + + SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [col1#x AS i#x, col2#x AS j#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x, col2#x] +: +- Project [i#x, (j#x + 1) AS (j + 1)#x] +: +- Join Inner, (i#x = (i#x + 1)) +: :- SubqueryAlias t2 +: : +- Union false, false +: : :- Project [2 AS i#x] +: : : +- OneRowRelation +: : +- Project [3 AS i#x] +: : +- OneRowRelation +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [i#x, j#x], false ++- Project [i#x, j#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [i#x, j#x], false, false + + -- !query CREATE TABLE tree( id INTEGER, @@ -122,6 +781,111 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d +- LocalRelation [col1#x, col2#x] +-- !query +WITH RECURSIVE t(id, path) AS ( + VALUES(1,cast(array() as array)) +UNION ALL + SELECT tree.id, t.path || array(tree.id) + FROM tree JOIN t ON (tree.parent_id = t.id) +) +SELECT t1.*, t2.* FROM t AS t1 JOIN t AS t2 ON + (t1.path[0] = t2.path[0] AND + size(t1.path) = 1 AND + size(t2.path) > 1) + ORDER BY t1.id, t2.id +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [col1#x AS id#x, col2#x AS path#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x, col2#x] +: +- Project [id#x, concat(path#x, array(id#x)) AS concat(path, array(id))#x] +: +- Join Inner, (parent_id#x = id#x) +: :- SubqueryAlias spark_catalog.default.tree +: : +- Relation spark_catalog.default.tree[id#x,parent_id#x] parquet +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [id#x, path#x], false ++- Sort [id#x ASC NULLS FIRST, id#x ASC NULLS FIRST], true + +- Project [id#x, path#x, id#x, path#x] + +- 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 + +- SubqueryAlias t2 + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [id#x, path#x], false, false + + +-- !query +WITH RECURSIVE t(id, path) AS ( + VALUES(1,cast(array() as array)) +UNION ALL + SELECT tree.id, t.path || array(tree.id) + FROM tree JOIN t ON (tree.parent_id = t.id) +) +SELECT t1.id, count(*) FROM t AS t1 JOIN t AS t2 ON + (t1.path[0] = t2.path[0] AND + size(t1.path) = 1 AND + size(t2.path) > 1) + GROUP BY t1.id + ORDER BY t1.id +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [col1#x AS id#x, col2#x AS path#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x, col2#x] +: +- Project [id#x, concat(path#x, array(id#x)) AS concat(path, array(id))#x] +: +- Join Inner, (parent_id#x = id#x) +: :- SubqueryAlias spark_catalog.default.tree +: : +- Relation spark_catalog.default.tree[id#x,parent_id#x] parquet +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [id#x, path#x], false ++- Sort [id#x ASC NULLS FIRST], true + +- Aggregate [id#x], [id#x, count(1) AS count(1)#xL] + +- 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 + +- SubqueryAlias t2 + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [id#x, path#x], false, false + + +-- !query +WITH RECURSIVE t(id, path) AS ( + VALUES(1,cast(array() as array)) +UNION ALL + SELECT tree.id, t.path || array(tree.id) + FROM tree JOIN t ON (tree.parent_id = t.id) +) +SELECT t1.id, t2.path, struct(t2.*) FROM t AS t1 JOIN t AS t2 ON +(t1.id=t2.id) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [col1#x AS id#x, col2#x AS path#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x, col2#x] +: +- Project [id#x, concat(path#x, array(id#x)) AS concat(path, array(id))#x] +: +- Join Inner, (parent_id#x = id#x) +: :- SubqueryAlias spark_catalog.default.tree +: : +- Relation spark_catalog.default.tree[id#x,parent_id#x] parquet +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [id#x, path#x], false ++- Project [id#x, path#x, struct(id, id#x, path, path#x) AS struct(id, path)#x] + +- Join Inner, (id#x = id#x) + :- SubqueryAlias t1 + : +- SubqueryAlias t + : +- CTERelationRef xxxx, true, [id#x, path#x], false, false + +- SubqueryAlias t2 + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [id#x, path#x], false, false + + -- !query create table graph( f int, t int, label string ) USING parquet -- !query analysis @@ -142,6 +906,194 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d +- LocalRelation [col1#x, col2#x, col3#x] +-- !query +with recursive search_graph(f, t, label, path, cycle) as ( + select *, array(struct(g.f, g.t)), false from graph g + union all + select g.*, path || array(struct(g.f, g.t)), array_contains(path, struct(g.f, g.t)) + from graph g, search_graph sg + where g.f = sg.t and not cycle +) +select * from search_graph +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias search_graph +: +- Project [f#x AS f#x, t#x AS t#x, label#x AS label#x, array(struct(f, t))#x AS path#x, false#x AS cycle#x] +: +- UnionLoop xxxx +: :- Project [f#x, t#x, label#x, array(struct(f, f#x, t, t#x)) AS array(struct(f, t))#x, false AS false#x] +: : +- SubqueryAlias g +: : +- SubqueryAlias spark_catalog.default.graph +: : +- Relation spark_catalog.default.graph[f#x,t#x,label#x] parquet +: +- Project [f#x, t#x, label#x, concat(path#x, array(struct(f, f#x, t, t#x))) AS concat(path, array(struct(f, t)))#x, array_contains(path#x, struct(f, f#x, t, t#x)) AS array_contains(path, struct(f, t))#x] +: +- Filter ((f#x = t#x) AND NOT cycle#x) +: +- Join Inner +: :- SubqueryAlias g +: : +- SubqueryAlias spark_catalog.default.graph +: : +- Relation spark_catalog.default.graph[f#x,t#x,label#x] parquet +: +- SubqueryAlias sg +: +- SubqueryAlias search_graph +: +- UnionLoopRef xxxx, [f#x, t#x, label#x, path#x, cycle#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 + + +-- !query +with recursive search_graph(f, t, label, path, cycle) as ( + select *, array(struct(g.f, g.t)), false from graph g + union all + select g.*, path || array(struct(g.f, g.t)), array_contains(path, struct(g.f, g.t)) + from graph g, search_graph sg + where g.f = sg.t and not cycle +) +select * from search_graph order by path +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias search_graph +: +- Project [f#x AS f#x, t#x AS t#x, label#x AS label#x, array(struct(f, t))#x AS path#x, false#x AS cycle#x] +: +- UnionLoop xxxx +: :- Project [f#x, t#x, label#x, array(struct(f, f#x, t, t#x)) AS array(struct(f, t))#x, false AS false#x] +: : +- SubqueryAlias g +: : +- SubqueryAlias spark_catalog.default.graph +: : +- Relation spark_catalog.default.graph[f#x,t#x,label#x] parquet +: +- Project [f#x, t#x, label#x, concat(path#x, array(struct(f, f#x, t, t#x))) AS concat(path, array(struct(f, t)))#x, array_contains(path#x, struct(f, f#x, t, t#x)) AS array_contains(path, struct(f, t))#x] +: +- Filter ((f#x = t#x) AND NOT cycle#x) +: +- Join Inner +: :- SubqueryAlias g +: : +- SubqueryAlias spark_catalog.default.graph +: : +- Relation spark_catalog.default.graph[f#x,t#x,label#x] parquet +: +- SubqueryAlias sg +: +- SubqueryAlias search_graph +: +- UnionLoopRef xxxx, [f#x, t#x, label#x, path#x, cycle#x], false ++- 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 + + +-- !query +WITH RECURSIVE + y (id) AS (VALUES (1)), + x (id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5) +SELECT * FROM x +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, false +: +- SubqueryAlias y +: +- Project [col1#x AS id#x] +: +- LocalRelation [col1#x] +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias x +: +- Project [id#x AS id#x] +: +- UnionLoop xxxx +: :- Project [id#x] +: : +- SubqueryAlias y +: : +- CTERelationRef xxxx, true, [id#x], false, false +: +- Project [(id#x + 1) AS (id + 1)#x] +: +- Filter (id#x < 5) +: +- SubqueryAlias x +: +- UnionLoopRef xxxx, [id#x], false ++- Project [id#x] + +- SubqueryAlias x + +- CTERelationRef xxxx, true, [id#x], false, false + + +-- !query +WITH RECURSIVE + x(id) AS + (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), + y(id) AS + (SELECT * FROM x UNION ALL SELECT * FROM x), + z(id) AS + (SELECT * FROM x UNION ALL SELECT id+1 FROM z WHERE id < 10) + SELECT * FROM z +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias x +: +- Project [1#x AS id#x] +: +- UnionLoop xxxx +: :- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Project [(id#x + 1) AS (id + 1)#x] +: +- Filter (id#x < 3) +: +- SubqueryAlias x +: +- UnionLoopRef xxxx, [id#x], false +:- CTERelationDef xxxx, false, false +: +- SubqueryAlias y +: +- Project [id#x AS id#x] +: +- Union false, false +: :- Project [id#x] +: : +- SubqueryAlias x +: : +- CTERelationRef xxxx, true, [id#x], false, false +: +- Project [id#x] +: +- SubqueryAlias x +: +- CTERelationRef xxxx, true, [id#x], false, false +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias z +: +- Project [id#x AS id#x] +: +- UnionLoop xxxx +: :- Project [id#x] +: : +- SubqueryAlias x +: : +- CTERelationRef xxxx, true, [id#x], false, false +: +- Project [(id#x + 1) AS (id + 1)#x] +: +- Filter (id#x < 10) +: +- SubqueryAlias z +: +- UnionLoopRef xxxx, [id#x], false ++- Project [id#x] + +- SubqueryAlias z + +- CTERelationRef xxxx, true, [id#x], false, false + + +-- !query +WITH RECURSIVE + x(id) AS + (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), + y(id) AS + (SELECT * FROM x UNION ALL SELECT * FROM x), + z(id) AS + (SELECT * FROM y UNION ALL SELECT id+1 FROM z WHERE id < 10) + SELECT * FROM z +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias x +: +- Project [1#x AS id#x] +: +- UnionLoop xxxx +: :- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Project [(id#x + 1) AS (id + 1)#x] +: +- Filter (id#x < 3) +: +- SubqueryAlias x +: +- UnionLoopRef xxxx, [id#x], false +:- CTERelationDef xxxx, false, false +: +- SubqueryAlias y +: +- Project [id#x AS id#x] +: +- Union false, false +: :- Project [id#x] +: : +- SubqueryAlias x +: : +- CTERelationRef xxxx, true, [id#x], false, false +: +- Project [id#x] +: +- SubqueryAlias x +: +- CTERelationRef xxxx, true, [id#x], false, false +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias z +: +- Project [id#x AS id#x] +: +- UnionLoop xxxx +: :- Project [id#x] +: : +- SubqueryAlias y +: : +- CTERelationRef xxxx, true, [id#x], false, false +: +- Project [(id#x + 1) AS (id + 1)#x] +: +- Filter (id#x < 10) +: +- SubqueryAlias z +: +- UnionLoopRef xxxx, [id#x], false ++- Project [id#x] + +- SubqueryAlias z + +- CTERelationRef xxxx, true, [id#x], false, false + + -- !query CREATE TABLE y (a INTEGER) USING parquet -- !query analysis @@ -165,6 +1117,117 @@ DropTable false, false +- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.y +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x)\n\tSELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x)\n\tSELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 75, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x)\n\tSELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x)\n\tSELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT n FROM x) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 57, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT n FROM x)\n\tSELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`n`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "n" + } ] +} + + -- !query CREATE TABLE y (a INTEGER) USING parquet -- !query analysis @@ -181,11 +1244,327 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d +- OneRowRelation +-- !query +WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 + UNION ALL + SELECT x.n+1 FROM y LEFT JOIN x ON x.n = y.a WHERE n < 10) +SELECT * FROM x +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 95, + "stopIndex" : 95, + "fragment" : "x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 + UNION ALL + SELECT x.n+1 FROM x RIGHT JOIN y ON x.n = y.a WHERE n < 10) +SELECT * FROM x +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 83, + "stopIndex" : 83, + "fragment" : "x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 + UNION ALL + SELECT x.n+1 FROM x FULL JOIN y ON x.n = y.a WHERE n < 10) +SELECT * FROM x +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 83, + "stopIndex" : 83, + "fragment" : "x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x + WHERE n IN (SELECT * FROM x)) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 114, + "stopIndex" : 114, + "fragment" : "x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT count(*) FROM x) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 65, + "stopIndex" : 65, + "fragment" : "x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT sum(n) FROM x) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 63, + "stopIndex" : 63, + "fragment" : "x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x ORDER BY 1) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 90, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x ORDER BY 1)\n SELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x LIMIT 10 OFFSET 1) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 97, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x LIMIT 10 OFFSET 1)\n SELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(id) AS (values (1) + UNION ALL + SELECT (SELECT * FROM x) FROM x WHERE id < 5 +) SELECT * FROM x +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 77, + "stopIndex" : 77, + "fragment" : "x" + } ] +} + + +-- !query +WITH RECURSIVE + x (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM y WHERE id < 5), + y (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 5) +SELECT * FROM x +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`id`", + "proposal" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 73, + "stopIndex" : 74, + "fragment" : "id" + } ] +} + + +-- !query +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + UNION ALL + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836" +} + + +-- !query +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + SELECT * FROM + (SELECT i+1 FROM foo WHERE i < 10 + UNION ALL + SELECT i+1 FROM foo WHERE i < 5) AS t +) SELECT * FROM foo +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836" +} + + +-- !query +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + EXCEPT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836" +} + + +-- !query +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + INTERSECT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836" +} + + +-- !query +WITH RECURSIVE foo(i) AS + (SELECT i FROM (VALUES(1),(2)) t(i) + UNION ALL + SELECT cast((i+1) AS decimal(10,0)) FROM foo WHERE i < 10) +SELECT * FROM foo +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.DATA_TYPE", + "sqlState" : "42836", + "messageParameters" : { + "fromDataType" : "\"INT\"", + "toDataType" : "\"DECIMAL(10,0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 122, + "stopIndex" : 124, + "fragment" : "foo" + } ] +} + + +-- !query +WITH RECURSIVE foo(i) AS + (SELECT cast(i as decimal(3,0)) FROM (VALUES(1),(2)) t(i) + UNION ALL + SELECT cast((i+1) AS decimal(10,0)) FROM foo WHERE i < 10) +SELECT * FROM foo +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.DATA_TYPE", + "sqlState" : "42836", + "messageParameters" : { + "fromDataType" : "\"DECIMAL(3,0)\"", + "toDataType" : "\"DECIMAL(10,0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 144, + "stopIndex" : 146, + "fragment" : "foo" + } ] +} + + -- !query with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [42#x AS foo#x] : +- Project [42 AS 42#x] @@ -194,7 +1573,46 @@ WithCTE +- SubqueryAlias q +- Project [foo#x] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [foo#x], false + +- CTERelationRef xxxx, true, [foo#x], false, false + + +-- !query +WITH RECURSIVE t(j) AS ( + WITH RECURSIVE s(i) AS ( + VALUES (1) + UNION ALL + SELECT i+1 FROM s WHERE i < 10 + ) + SELECT i FROM s + UNION ALL + SELECT j+1 FROM t WHERE j < 10 +) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias s +: +- Project [col1#x AS i#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(i#x + 1) AS (i + 1)#x] +: +- Filter (i#x < 10) +: +- SubqueryAlias s +: +- UnionLoopRef xxxx, [i#x], false +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias t +: +- Project [i#x AS j#x] +: +- UnionLoop xxxx +: :- Project [i#x] +: : +- SubqueryAlias s +: : +- CTERelationRef xxxx, true, [i#x], false, false +: +- Project [(j#x + 1) AS (j + 1)#x] +: +- Filter (j#x < 10) +: +- SubqueryAlias t +: +- UnionLoopRef xxxx, [j#x], false ++- Project [j#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [j#x], false, false -- !query @@ -207,11 +1625,11 @@ WITH outermost(x) AS ( SELECT * FROM outermost ORDER BY 1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias innermost : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias outermost : +- Project [1#x AS x#x] : +- Distinct @@ -222,13 +1640,13 @@ WithCTE : +- Union false, false : :- Project [2#x] : : +- SubqueryAlias innermost -: : +- CTERelationRef xxxx, true, [2#x], false +: : +- CTERelationRef xxxx, true, [2#x], false, 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 + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -257,6 +1675,68 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +WITH RECURSIVE outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false, false +: +- SubqueryAlias innermost +: +- Project [2 AS 2#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false, true +: +- SubqueryAlias outermost +: +- Project [1#x AS x#x] +: +- UnionLoop xxxx +: :- Distinct +: : +- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Except false +: :- Distinct +: : +- Union false, false +: : :- Project [x#x] +: : : +- SubqueryAlias outermost +: : : +- UnionLoopRef xxxx, [x#x], false +: : +- Project [2#x] +: : +- SubqueryAlias innermost +: : +- CTERelationRef xxxx, true, [2#x], false, false +: +- UnionLoopRef xxxx, [x#x], true ++- Sort [x#x ASC NULLS FIRST], true + +- Project [x#x] + +- SubqueryAlias outermost + +- CTERelationRef xxxx, true, [x#x], false, false + + +-- !query +WITH RECURSIVE outermost(x) AS ( + WITH innermost as (SELECT 2 FROM outermost) -- fail + SELECT * FROM innermost + UNION SELECT * from outermost +) +SELECT * FROM outermost ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`outermost`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 77, + "fragment" : "outermost" + } ] +} + + -- !query CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i AS string) || ' v' AS v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i) -- !query analysis @@ -412,13 +1892,13 @@ org.apache.spark.sql.catalyst.parser.ParseException with ordinality as (select 1 as x) select * from ordinality -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias ordinality : +- Project [1 AS x#x] : +- OneRowRelation +- Project [x#x] +- SubqueryAlias ordinality - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -453,13 +1933,13 @@ with test as (select 42) insert into test select * from test InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test], Append, `spark_catalog`.`default`.`test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test), [i] +- Project [cast(42#x as int) AS i#x] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias test : +- Project [42 AS 42#x] : +- OneRowRelation +- Project [42#x] +- SubqueryAlias test - +- CTERelationRef xxxx, true, [42#x], false + +- CTERelationRef xxxx, true, [42#x], false, false -- !query @@ -477,6 +1957,24 @@ DropTable false, false +- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test +-- !query +DROP VIEW nums +-- !query analysis +DropTempViewCommand nums + + +-- !query +DROP VIEW vsubdepartment +-- !query analysis +DropTempViewCommand vsubdepartment + + +-- !query +DROP VIEW sums_1_100 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`sums_1_100`, false, true, false + + -- !query DROP TABLE department -- !query analysis 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 e75c7946ef765..bb15ef0b25268 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 @@ -2048,13 +2048,13 @@ Project [1 AS 1#x] WITH v1 AS (SELECT var1 AS c1) SELECT c1 AS `1` FROM v1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias v1 : +- Project [variablereference(system.session.var1=1) AS c1#x] : +- OneRowRelation +- Project [c1#x AS 1#x] +- SubqueryAlias v1 - +- CTERelationRef xxxx, true, [c1#x], false + +- CTERelationRef xxxx, true, [c1#x], false, false -- !query 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 cab83b2649974..2241e6ad7b2a5 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 @@ -101,7 +101,7 @@ WHERE a.bonus_amt > 30 WHERE a.emp_name = b.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias bonus_cte : +- Project [emp_name#x, bonus_amt#x] : +- Filter exists#x [emp_name#x] @@ -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 + : +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], false, false +- SubqueryAlias a +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x,bonus_amt#x]) @@ -162,7 +162,7 @@ WHERE EXISTS (SELECT * WHERE bonus.emp_name = a.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias emp_cte : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- Filter ((id#x >= 100) AND (id#x <= 300)) @@ -172,7 +172,7 @@ WithCTE : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- SubqueryAlias EMP : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias dept_cte : +- Project [dept_id#x, dept_name#x, state#x] : +- Filter (dept_id#x = 10) @@ -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 + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false : +- SubqueryAlias b : +- SubqueryAlias dept_cte - : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], 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] @@ -225,7 +225,7 @@ WHERE e.dept_id = d.dept_id WHERE e.emp_name = a.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias emp_cte : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- Filter ((id#x >= 100) AND (id#x <= 300)) @@ -235,7 +235,7 @@ WithCTE : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- SubqueryAlias EMP : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias dept_cte : +- Project [dept_id#x, dept_name#x, state#x] : +- Filter (dept_id#x = 10) @@ -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 + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false : +- SubqueryAlias b : +- SubqueryAlias dept_cte - : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], 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 + : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false +- SubqueryAlias d +- SubqueryAlias dept +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) @@ -299,7 +299,7 @@ WHERE EXISTS (SELECT dept_id, GROUP BY emp_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias empdept : +- Project [id#x, salary#x, emp_name#x, dept_id#x] : +- Filter id#x IN (100,200) @@ -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 + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], 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] @@ -352,7 +352,7 @@ WHERE NOT EXISTS (SELECT dept_id, GROUP BY emp_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias empdept : +- Project [id#x, salary#x, emp_name#x, dept_id#x] : +- Filter id#x IN (100,200) @@ -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 + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], 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 1717e553f5c3c..9ced2e8e6a8e4 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 @@ -307,7 +307,7 @@ FROM ( on cte1.t1b = cte2.t1b) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter named_struct(t1b, t1b#x, t1d, t1d#xL) IN (list#x [t1c#x]) @@ -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 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], 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 6d0a944bfcfe2..2b0a44a6f0563 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 @@ -123,7 +123,7 @@ WHERE t1b IN (SELECT cte1.t1b WHERE cte1.t1b > 0) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1a#x = val1a) @@ -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 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], 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] @@ -177,7 +177,7 @@ GROUP BY t1a, t1b, t1c HAVING t1c IS NOT NULL -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- SubqueryAlias t1 @@ -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 + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : : +- Project [t1b#x] : : : +- Filter (cast(t1b#x as int) > 5) : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : +- Intersect false : : :- Project [t1b#x] : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : +- Project [t1b#x] : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : +- Project [t1b#x] : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], 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] @@ -250,7 +250,7 @@ WHERE t1c IN ON cte1.t1d > cte6.t1d) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] : +- SubqueryAlias t1 @@ -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 + : : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : : : : +- SubqueryAlias cte2 : : : : : +- SubqueryAlias cte1 - : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : : : +- SubqueryAlias cte3 : : : : +- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : : +- SubqueryAlias cte4 : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : +- SubqueryAlias cte5 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : +- SubqueryAlias cte6 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], 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] @@ -316,7 +316,7 @@ FROM (SELECT * ON cte1.t1b = cte4.t1b) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1b#x IN (list#x []) AND (t1a#x = val1b)) @@ -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 + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : +- SubqueryAlias cte2 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : +- SubqueryAlias cte3 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte4 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false -- !query @@ -393,7 +393,7 @@ WHERE t1b IN GROUP BY t1b -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1h#x] : +- Filter t1a#x IN (list#x [t1b#x]) @@ -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 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false -- !query @@ -453,7 +453,7 @@ FROM ( ) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x] : +- Filter (t1b#x IN (list#x [t1c#x]) AND (t1a#x = val1b)) @@ -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 + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false : : +- SubqueryAlias cte2 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false : +- SubqueryAlias cte3 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false +- SubqueryAlias cte4 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false -- !query @@ -515,7 +515,7 @@ FROM (SELECT cte1.t1a, GROUP BY s.t1b -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter t1b#x IN (list#x [t1c#x]) @@ -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 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false -- !query @@ -569,7 +569,7 @@ WHERE s.t1b IN ON t1.t1a = cte1.t1a) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter t1b#x IN (list#x [t1c#x]) @@ -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 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias s +- Project [t1b#x] +- Join LeftOuter, (t1b#x = t1b#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false -- !query @@ -627,7 +627,7 @@ WHERE t1b NOT IN (SELECT cte1.t1b t1c > 10 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1a#x = val1d) @@ -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 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], 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] @@ -683,7 +683,7 @@ WHERE t1b NOT IN ORDER BY t1c DESC -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] : +- Filter NOT t1d#xL IN (list#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 + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false : : : +- SubqueryAlias cte2 : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false : : +- SubqueryAlias cte3 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false : +- SubqueryAlias cte4 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], 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 5265b6163bd35..f2c1dd9a49fe9 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 @@ -617,13 +617,13 @@ SELECT c1, (WITH t AS (SELECT 1 AS a) SELECT a + c1 FROM t) FROM t1 -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [1 AS a#x] : : +- OneRowRelation : +- Project [(a#x + outer(c1#x)) AS (a + outer(t1.c1))#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [a#x], false +: +- CTERelationRef xxxx, true, [a#x], 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] @@ -636,7 +636,7 @@ SELECT c1, (WITH t AS (SELECT * FROM t2 WHERE c1 = t1.c1) SELECT SUM(c2) FROM t) -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- Filter (c1#x = outer(c1#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 +: +- CTERelationRef xxxx, true, [c1#x, c2#x], 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] @@ -664,7 +664,7 @@ SELECT c1, ( -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t3 : : +- Project [(c1#x + 1) AS c1#x, (c2#x + 1) AS c2#x] : : +- SubqueryAlias t2 @@ -672,15 +672,15 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] : : +- SubqueryAlias t2 : : +- LocalRelation [c1#x, c2#x] -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t4 : : +- Project [c1#x, c2#x] : : +- Filter (outer(c1#x) = c1#x) : : +- SubqueryAlias t3 -: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : +- SubqueryAlias t4 -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], 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] @@ -697,7 +697,7 @@ SELECT c1, ( -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- SubqueryAlias t2 @@ -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 +: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false : +- Project [c2#x, c1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], 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] @@ -732,7 +732,7 @@ SELECT * FROM t1 WHERE c1 > ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias v : +- Project [c1#x, c2#x] : +- SubqueryAlias t2 @@ -743,7 +743,7 @@ WithCTE +- Project [c1#x, c2#x] +- Filter (cast(c1#x as bigint) > scalar-subquery#x [c1#x]) : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- SubqueryAlias t2 @@ -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 + : : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false : +- SubqueryAlias v - : +- CTERelationRef xxxx, true, [c1#x, c2#x], false + : +- CTERelationRef xxxx, true, [c1#x, c2#x], 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] @@ -771,7 +771,7 @@ WITH t AS (SELECT 1 AS a) SELECT c1, (SELECT a FROM t WHERE a = c1) FROM t1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS a#x] : +- OneRowRelation @@ -779,7 +779,7 @@ WithCTE : +- Project [a#x] : +- Filter (a#x = outer(c1#x)) : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [a#x], false + : +- CTERelationRef xxxx, true, [a#x], 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] @@ -1019,7 +1019,7 @@ SELECT (SELECT sum(1) FROM T WHERE a = col OR upper(col)= 'Y') FROM (SELECT null as col) as foo -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T : +- Project [1 AS a#x] : +- OneRowRelation @@ -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 + : +- CTERelationRef xxxx, true, [a#x], false, 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 ceca433a1c915..d92236d5d9e65 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 @@ -874,7 +874,7 @@ WITH temp AS ( SELECT t1.b FROM temp t1 JOIN temp t2 ON t1.b = t2.b -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias temp : +- ScriptTransformation cat, [b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) : +- Project [a#x] @@ -888,10 +888,10 @@ WithCTE +- Join Inner, (b#x = b#x) :- SubqueryAlias t1 : +- SubqueryAlias temp - : +- CTERelationRef xxxx, true, [b#x], false + : +- CTERelationRef xxxx, true, [b#x], false, false +- SubqueryAlias t2 +- SubqueryAlias temp - +- CTERelationRef xxxx, true, [b#x], false + +- CTERelationRef xxxx, true, [b#x], 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 97410d3cdd369..e996190fc5a1c 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 @@ -817,12 +817,12 @@ FROM t1 FULL OUTER JOIN t2 USING (key) WHERE t1.key NOT LIKE 'bb.%' -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [key#x] : +- SubqueryAlias t : +- LocalRelation [key#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [key#x] : +- SubqueryAlias t @@ -833,6 +833,6 @@ WithCTE +- Project [coalesce(key#x, key#x) AS key#x, key#x, key#x, key#x] +- Join FullOuter, (key#x = key#x) :- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [key#x], false + : +- CTERelationRef xxxx, true, [key#x], false, false +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [key#x], false + +- CTERelationRef xxxx, true, [key#x], false, 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 new file mode 100644 index 0000000000000..f70fb4dc8b763 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql @@ -0,0 +1,402 @@ +--CONFIG_DIM2 spark.sql.adaptive.enabled=false +--CONFIG_DIM2 spark.sql.adaptive.enabled=true + +-- fails due to recursion isn't allowed without RECURSIVE keyword +WITH r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r; + +-- basic recursion +WITH RECURSIVE r AS ( + SELECT 0 AS level + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r; + +-- basic recursion with subquery column alias +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r; + +-- using string column in recursion +WITH RECURSIVE r(c) AS ( + SELECT 'a' + UNION ALL + SELECT c || char(ascii(substr(c, -1)) + 1) FROM r WHERE LENGTH(c) < 10 +) +SELECT * FROM r; + +-- unlimited recursion fails at spark.sql.cte.recursion.level.limits level +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r; + +-- terminate recursion with LIMIT +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r LIMIT 10; + +-- terminate recursion with UNION +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION + SELECT (level + 1) % 10 FROM r +) +SELECT * FROM r; + +-- unlimited recursion fails because using LIMIT to terminate recursion only works where it can be +-- pushed down to recursion +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT level, level FROM r ORDER BY 1 LIMIT 10; + +-- fails because recursion doesn't follow the expected form +WITH RECURSIVE r(level) AS ( + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r; + +-- fails because recursion doesn't follow the expected form +WITH RECURSIVE r(level) AS ( + SELECT level + 1 FROM r WHERE level < 9 + UNION ALL + VALUES 0 +) +SELECT * FROM r; + +-- fails because a recursive query should contain UNION ALL or UNION combinator +WITH RECURSIVE r(level) AS ( + VALUES 0 + INTERSECT + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r; + +-- recursive reference is not allowed in a subquery expression +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE (SELECT SUM(level) FROM r) < 9 +) +SELECT * FROM r; + +-- recursive reference is not allowed in a nested CTE +WITH RECURSIVE + t1 AS ( + SELECT 1 AS level + UNION ( + WITH t2 AS (SELECT level + 1 FROM t1 WHERE level < 10) + SELECT * FROM t2 + ) + ) +SELECT * FROM t1; + +-- recursive reference and conflicting outer CTEs are not allowed in a nested CTE +SET spark.sql.legacy.ctePrecedencePolicy=CORRECTED; +WITH + t1 AS (SELECT 1), + t2 AS ( + WITH RECURSIVE + t1 AS ( + SELECT 1 AS level + UNION ( + WITH t3 AS (SELECT level + 1 FROM t1 WHERE level < 10) + SELECT * FROM t3 + ) + ) + SELECT * FROM t1 + ) +SELECT * FROM t2; +SET spark.sql.legacy.ctePrecedencePolicy=EXCEPTION; + +-- recursive reference can't be used multiple times in a recursive term +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT r1.level + 1, r1.data + FROM r AS r1 + JOIN r AS r2 ON r2.data = r1.data + WHERE r1.level < 9 +) +SELECT * FROM r; + +-- recursive reference is allowed on left side of a left outer join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + LEFT OUTER JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r; + +-- recursive reference is not allowed on right side of a left outer join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM ( + SELECT 0 AS data + ) AS t + LEFT OUTER JOIN r ON r.data = t.data + WHERE r.level < 9 +) +SELECT * FROM r; + +-- recursive reference is allowed on right side of a right outer join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM ( + SELECT 0 AS data + ) AS t + RIGHT OUTER JOIN r ON r.data = t.data + WHERE r.level < 9 +) +SELECT * FROM r; + +-- recursive reference is not allowed on left side of a right outer join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + RIGHT OUTER JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r; + +-- recursive reference is not allowed in a full outer join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + FULL OUTER JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r; + +-- recursive reference is allowed on left side of a left semi join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + LEFT SEMI JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r; + +-- recursive reference is not allowed on right side of a left semi join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, data + FROM ( + SELECT 0 AS level, 0 AS data + ) AS t + LEFT SEMI JOIN r ON r.data = t.data AND r.level < 9 +) +SELECT * FROM r; + +-- recursive reference is allowed on left side of a left anti join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + LEFT ANTI JOIN ( + SELECT -1 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r; + +-- recursive reference is not allowed on right side of a left anti join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, data + FROM ( + SELECT 0 AS level, 0 AS data + ) AS t + LEFT ANTI JOIN r ON r.data = t.data AND r.level < 9 +) +SELECT * FROM r; + +-- recursive reference is not allowed in an aggregate +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 1L) + UNION ALL + SELECT MAX(level) + 1, SUM(data) FROM r WHERE level < 9 +) +SELECT * FROM r; + +-- recursion is allowed in simple commands +CREATE TEMPORARY VIEW rv AS +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r; + +SELECT * FROM rv; + +DROP VIEW rv; + +-- recursion is allowed in simple commands 2 +CREATE TABLE rt(level INT) USING csv; + +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +INSERT INTO rt SELECT * FROM r; + +SELECT * from rt; + +DROP TABLE rt; + +-- recursion is not allowed in multi commands +CREATE TABLE rt2(level INT) USING csv; + +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +FROM r +INSERT INTO rt2 SELECT * +INSERT INTO rt2 SELECT *; + +DROP TABLE rt2; + +-- multiple recursive CTEs +WITH RECURSIVE + r1 AS ( + SELECT 0 AS level + UNION ALL + SELECT level + 1 FROM r1 WHERE level < 9 + ), + r2 AS ( + SELECT 10 AS level + UNION ALL + SELECT level + 1 FROM r2 WHERE level < 19 + ) +SELECT * +FROM r1 +JOIN r2 ON r2.level = r1.level + 10; + +-- multiple uses of recursive CTEs +WITH RECURSIVE r AS ( + SELECT 0 AS level + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * +FROM r AS r1 +JOIN r AS r2 ON r2.level = r1.level; + +-- recursive cte nested into recursive cte as anchor +WITH RECURSIVE r2 AS ( + WITH RECURSIVE r1 AS ( + SELECT 0 AS innerlevel + UNION ALL + SELECT innerlevel + 1 FROM r1 WHERE innerlevel < 3 + ) + SELECT 0 AS outerlevel, innerlevel FROM r1 + UNION ALL + SELECT outerlevel + 1, innerlevel FROM r2 WHERE outerlevel < 3 +) +SELECT * FROM r2; + +-- name collision of nested CTEs (the outer CTE is not recursive) +WITH RECURSIVE r(level) AS ( + WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 + ) + SELECT * FROM r + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 +) +SELECT * FROM r; + +-- name collision of nested CTEs (the outer CTE is recursive) +WITH RECURSIVE r(level) AS ( + (WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 + ) + SELECT * FROM r) + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 +) +SELECT * FROM r; + +-- routes represented here is as follows: +-- +-- New York<--->Boston +-- | ∧ +-- ∨ | +-- Washington---+ +-- | +-- ∨ +-- Raleigh +CREATE TEMPORARY VIEW routes(origin, destination) AS VALUES + ('New York', 'Washington'), + ('New York', 'Boston'), + ('Boston', 'New York'), + ('Washington', 'Boston'), + ('Washington', 'Raleigh'); + +-- handling cycles that could cause infinite recursion +WITH RECURSIVE destinations_from_new_york AS ( + SELECT 'New York' AS destination, ARRAY('New York') AS path, 0 AS length + UNION ALL + SELECT r.destination, CONCAT(d.path, ARRAY(r.destination)), d.length + 1 + FROM routes AS r + JOIN destinations_from_new_york AS d ON d.destination = r.origin AND NOT ARRAY_CONTAINS(d.path, r.destination) +) +SELECT * FROM destinations_from_new_york; + +DROP VIEW routes; + +-- Fibonacci numbers +WITH RECURSIVE fibonacci AS ( + VALUES (0, 1) AS t(a, b) + UNION ALL + SELECT b, a + b FROM fibonacci WHERE a < 10 +) +SELECT a FROM fibonacci ORDER BY a; diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain.sql b/sql/core/src/test/resources/sql-tests/inputs/explain.sql index 698ca009b4ffb..96dddafd82ac8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/explain.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/explain.sql @@ -96,6 +96,15 @@ EXPLAIN FORMATTED ) SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key; +-- Recursion +EXPLAIN FORMATTED + WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 + ) +SELECT * FROM r; + -- A spark plan which has innerChildren other than subquery EXPLAIN FORMATTED CREATE VIEW explain_view AS diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/with.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/with.sql index a3e0b15b582f5..48d1fb0b8ecfe 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/with.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/with.sql @@ -35,87 +35,113 @@ SELECT count(*) FROM ( -- WITH RECURSIVE -- sum of 1..100 --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE t(n) AS ( --- VALUES (1) ---UNION ALL --- SELECT n+1 FROM t WHERE n < 100 ---) ---SELECT sum(n) FROM t; +WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t WHERE n < 100 +) +SELECT sum(n) FROM t; --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE t(n) AS ( --- SELECT (VALUES(1)) ---UNION ALL --- SELECT n+1 FROM t WHERE n < 5 ---) ---SELECT * FROM t; +WITH RECURSIVE t(n) AS ( + SELECT (VALUES(1)) +UNION ALL + SELECT n+1 FROM t WHERE n < 5 +) +SELECT * FROM t; -- recursive view --- [SPARK-24497] Support recursive SQL query +-- [SPARK-28453] Support recursive view syntax +-- [ORIGINAL SQL] --CREATE RECURSIVE VIEW nums (n) AS -- VALUES (1) --UNION ALL -- SELECT n+1 FROM nums WHERE n < 5; --- ---SELECT * FROM nums; +CREATE TEMPORARY VIEW nums AS +WITH RECURSIVE nums (n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM nums WHERE n < 5 +) +SELECT * FROM nums; + +SELECT * FROM nums; --- [SPARK-24497] Support recursive SQL query +-- [SPARK-28453] Support recursive view syntax +-- [ORIGINAL SQL] --CREATE OR REPLACE RECURSIVE VIEW nums (n) AS -- VALUES (1) --UNION ALL -- SELECT n+1 FROM nums WHERE n < 6; --- ---SELECT * FROM nums; +CREATE OR REPLACE TEMPORARY VIEW nums AS +WITH RECURSIVE nums (n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM nums WHERE n < 6 +) +SELECT * FROM nums; + +SELECT * FROM nums; -- This is an infinite loop with UNION ALL, but not with UNION --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE t(n) AS ( --- SELECT 1 ---UNION --- SELECT 10-n FROM t) ---SELECT * FROM t; +WITH RECURSIVE t(n) AS ( + SELECT 1 +UNION + SELECT 10-n FROM t) +SELECT * FROM t; -- This'd be an infinite loop, but outside query reads only as much as needed --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE t(n) AS ( --- VALUES (1) ---UNION ALL --- SELECT n+1 FROM t) ---SELECT * FROM t LIMIT 10; +WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t) +SELECT * FROM t LIMIT 10; -- UNION case should have same property --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE t(n) AS ( --- SELECT 1 ---UNION --- SELECT n+1 FROM t) ---SELECT * FROM t LIMIT 10; +WITH RECURSIVE t(n) AS ( + SELECT 1 +UNION + SELECT n+1 FROM t) +SELECT * FROM t LIMIT 10; -- Test behavior with an unknown-type literal in the WITH -- [SPARK-28146] Support IS OF type predicate +-- [ORIGINAL SQL] --WITH q AS (SELECT 'foo' AS x) --SELECT x, x IS OF (text) AS is_text FROM q; +WITH q AS (SELECT 'foo' AS x) +SELECT x FROM q; --- [SPARK-24497] Support recursive SQL query -- [SPARK-28146] Support IS OF type predicate +-- [ORIGINAL SQL] --WITH RECURSIVE t(n) AS ( -- SELECT 'foo' --UNION ALL -- SELECT n || ' bar' FROM t WHERE length(n) < 20 --) --SELECT n, n IS OF (text) AS is_text FROM t; +WITH RECURSIVE t(n) AS ( + SELECT 'foo' +UNION ALL + SELECT n || ' bar' FROM t WHERE length(n) < 20 +) +SELECT n AS is_text FROM t; -- In a perfect world, this would work and resolve the literal as int ... -- but for now, we have to be content with resolving to text too soon. --- [SPARK-24497] Support recursive SQL query -- [SPARK-28146] Support IS OF type predicate +-- [ORIGINAL SQL] --WITH RECURSIVE t(n) AS ( -- SELECT '7' --UNION ALL -- SELECT n+1 FROM t WHERE n < 10 --) --SELECT n, n IS OF (int) AS is_int FROM t; +WITH RECURSIVE t(n) AS ( + SELECT '7' +UNION ALL + SELECT n+1 FROM t WHERE n < 10 +) +SELECT n FROM t; -- -- Some examples with a tree @@ -151,62 +177,58 @@ INSERT INTO department VALUES (7, 5, 'G'); -- extract all departments under 'A'. Result should be A, B, C, D and F --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE subdepartment AS ---( --- -- non recursive term --- SELECT name as root_name, * FROM department WHERE name = 'A' --- --- UNION ALL --- --- -- recursive term --- SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd --- WHERE d.parent_department = sd.id ---) ---SELECT * FROM subdepartment ORDER BY name; +WITH RECURSIVE subdepartment AS +( + -- non recursive term + SELECT name as root_name, * FROM department WHERE name = 'A' + + UNION ALL + + -- recursive term + SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id +) +SELECT * FROM subdepartment ORDER BY name; -- extract all departments under 'A' with "level" number --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE subdepartment(level, id, parent_department, name) AS ---( --- -- non recursive term --- SELECT 1, * FROM department WHERE name = 'A' --- --- UNION ALL --- --- -- recursive term --- SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd --- WHERE d.parent_department = sd.id ---) ---SELECT * FROM subdepartment ORDER BY name; +WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +( + -- non recursive term + SELECT 1, * FROM department WHERE name = 'A' + + UNION ALL + + -- recursive term + SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id +) +SELECT * FROM subdepartment ORDER BY name; -- extract all departments under 'A' with "level" number. -- Only shows level 2 or more --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE subdepartment(level, id, parent_department, name) AS ---( --- -- non recursive term --- SELECT 1, * FROM department WHERE name = 'A' --- --- UNION ALL --- --- -- recursive term --- SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd --- WHERE d.parent_department = sd.id ---) ---SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name; +WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +( + -- non recursive term + SELECT 1, * FROM department WHERE name = 'A' + + UNION ALL + + -- recursive term + SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id +) +SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name; -- "RECURSIVE" is ignored if the query has no self-reference --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE subdepartment AS ---( --- -- note lack of recursive UNION structure --- SELECT * FROM department WHERE name = 'A' ---) ---SELECT * FROM subdepartment ORDER BY name; +WITH RECURSIVE subdepartment AS +( + -- note lack of recursive UNION structure + SELECT * FROM department WHERE name = 'A' +) +SELECT * FROM subdepartment ORDER BY name; -- inside subqueries --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --SELECT count(*) FROM ( -- WITH RECURSIVE t(n) AS ( -- SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 500 @@ -218,48 +240,59 @@ INSERT INTO department VALUES (7, 5, 'G'); -- ) -- SELECT * FROM t WHERE n < 50000 -- ) AS t WHERE n < 100); +-- [NOTE] Decreased recursion depth to avoid stack overflow +SET spark.sql.cteRecursionLevelLimit=200; +SELECT count(*) FROM ( + WITH RECURSIVE t(n) AS ( + SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 200 + ) + SELECT * FROM t) AS t WHERE n < ( + SELECT count(*) FROM ( + WITH RECURSIVE t(n) AS ( + SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 100 + ) + SELECT * FROM t WHERE n < 50000 + ) AS t WHERE n < 100); +SET spark.sql.cteRecursionLevelLimit=100; -- use same CTE twice at different subquery levels --- [SPARK-24497] Support recursive SQL query ---WITH q1(x,y) AS ( --- SELECT hundred, sum(ten) FROM tenk1 GROUP BY hundred --- ) ---SELECT count(*) FROM q1 WHERE y > (SELECT sum(y)/100 FROM q1 qsub); +WITH q1(x,y) AS ( + SELECT hundred, sum(ten) FROM tenk1 GROUP BY hundred + ) +SELECT count(*) FROM q1 WHERE y > (SELECT sum(y)/100 FROM q1 qsub); -- via a VIEW --- [SPARK-24497] Support recursive SQL query ---CREATE TEMPORARY VIEW vsubdepartment AS --- WITH RECURSIVE subdepartment AS --- ( --- -- non recursive term --- SELECT * FROM department WHERE name = 'A' --- UNION ALL --- -- recursive term --- SELECT d.* FROM department AS d, subdepartment AS sd --- WHERE d.parent_department = sd.id --- ) --- SELECT * FROM subdepartment; --- ---SELECT * FROM vsubdepartment ORDER BY name; --- +CREATE TEMPORARY VIEW vsubdepartment AS + WITH RECURSIVE subdepartment AS + ( + -- non recursive term + SELECT * FROM department WHERE name = 'A' + UNION ALL + -- recursive term + SELECT d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id + ) + SELECT * FROM subdepartment; + +SELECT * FROM vsubdepartment ORDER BY name; + ---- Check reverse listing --SELECT pg_get_viewdef('vsubdepartment'::regclass); --SELECT pg_get_viewdef('vsubdepartment'::regclass, true); -- Another reverse-listing example --- [SPARK-24497] Support recursive SQL query ---CREATE VIEW sums_1_100 AS ---WITH RECURSIVE t(n) AS ( --- VALUES (1) ---UNION ALL --- SELECT n+1 FROM t WHERE n < 100 ---) ---SELECT sum(n) FROM t; --- ---\d+ sums_1_100 +CREATE VIEW sums_1_100 AS +WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t WHERE n < 100 +) +SELECT sum(n) AS sum FROM t; + +SELECT * FROM sums_1_100; -- corner case in which sub-WITH gets initialized first --- [SPARK-24497] Support recursive SQL query +-- [NOTE] Spark SQL doesn't support recursive reference in an inner CTE --with recursive q as ( -- select * from department -- union all @@ -268,7 +301,7 @@ INSERT INTO department VALUES (7, 5, 'G'); -- ) --select * from q limit 24; --- [SPARK-24497] Support recursive SQL query +-- [NOTE] Spark SQL doesn't support recursive reference in an inner CTE --with recursive q as ( -- select * from department -- union all @@ -282,15 +315,14 @@ INSERT INTO department VALUES (7, 5, 'G'); --select * from q limit 32; -- recursive term has sub-UNION --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE t(i,j) AS ( --- VALUES (1,2) --- UNION ALL --- SELECT t2.i, t.j+1 FROM --- (SELECT 2 AS i UNION ALL SELECT 3 AS i) AS t2 --- JOIN t ON (t2.i = t.i+1)) --- --- SELECT * FROM t; +WITH RECURSIVE t(i,j) AS ( + VALUES (1,2) + UNION ALL + SELECT t2.i, t.j+1 FROM + (SELECT 2 AS i UNION ALL SELECT 3 AS i) AS t2 + JOIN t ON (t2.i = t.i+1)) + + SELECT * FROM t; -- -- different tree example @@ -312,7 +344,7 @@ VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), -- -- get all paths from "second level" nodes to leaf nodes -- --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --WITH RECURSIVE t(id, path) AS ( -- VALUES(1,ARRAY[]::integer[]) --UNION ALL @@ -324,9 +356,20 @@ VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), -- array_upper(t1.path,1) = 1 AND -- array_upper(t2.path,1) > 1) -- ORDER BY t1.id, t2.id; +WITH RECURSIVE t(id, path) AS ( + VALUES(1,cast(array() as array)) +UNION ALL + SELECT tree.id, t.path || array(tree.id) + FROM tree JOIN t ON (tree.parent_id = t.id) +) +SELECT t1.*, t2.* FROM t AS t1 JOIN t AS t2 ON + (t1.path[0] = t2.path[0] AND + size(t1.path) = 1 AND + size(t2.path) > 1) + ORDER BY t1.id, t2.id; -- just count 'em --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --WITH RECURSIVE t(id, path) AS ( -- VALUES(1,ARRAY[]::integer[]) --UNION ALL @@ -339,9 +382,21 @@ VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), -- array_upper(t2.path,1) > 1) -- GROUP BY t1.id -- ORDER BY t1.id; +WITH RECURSIVE t(id, path) AS ( + VALUES(1,cast(array() as array)) +UNION ALL + SELECT tree.id, t.path || array(tree.id) + FROM tree JOIN t ON (tree.parent_id = t.id) +) +SELECT t1.id, count(*) FROM t AS t1 JOIN t AS t2 ON + (t1.path[0] = t2.path[0] AND + size(t1.path) = 1 AND + size(t2.path) > 1) + GROUP BY t1.id + ORDER BY t1.id; -- this variant tickled a whole-row-variable bug in 8.4devel --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --WITH RECURSIVE t(id, path) AS ( -- VALUES(1,ARRAY[]::integer[]) --UNION ALL @@ -350,6 +405,14 @@ VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), --) --SELECT t1.id, t2.path, t2 FROM t AS t1 JOIN t AS t2 ON --(t1.id=t2.id); +WITH RECURSIVE t(id, path) AS ( + VALUES(1,cast(array() as array)) +UNION ALL + SELECT tree.id, t.path || array(tree.id) + FROM tree JOIN t ON (tree.parent_id = t.id) +) +SELECT t1.id, t2.path, struct(t2.*) FROM t AS t1 JOIN t AS t2 ON +(t1.id=t2.id); -- -- test cycle detection @@ -366,7 +429,7 @@ insert into graph values (4, 5, 'arc 4 -> 5'), (5, 1, 'arc 5 -> 1'); --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --with recursive search_graph(f, t, label, path, cycle) as ( -- select *, array[row(g.f, g.t)], false from graph g -- union all @@ -375,9 +438,17 @@ insert into graph values -- where g.f = sg.t and not cycle --) --select * from search_graph; +with recursive search_graph(f, t, label, path, cycle) as ( + select *, array(struct(g.f, g.t)), false from graph g + union all + select g.*, path || array(struct(g.f, g.t)), array_contains(path, struct(g.f, g.t)) + from graph g, search_graph sg + where g.f = sg.t and not cycle +) +select * from search_graph; -- ordering by the path column has same effect as SEARCH DEPTH FIRST --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --with recursive search_graph(f, t, label, path, cycle) as ( -- select *, array[row(g.f, g.t)], false from graph g -- union all @@ -386,24 +457,31 @@ insert into graph values -- where g.f = sg.t and not cycle --) --select * from search_graph order by path; +with recursive search_graph(f, t, label, path, cycle) as ( + select *, array(struct(g.f, g.t)), false from graph g + union all + select g.*, path || array(struct(g.f, g.t)), array_contains(path, struct(g.f, g.t)) + from graph g, search_graph sg + where g.f = sg.t and not cycle +) +select * from search_graph order by path; -- -- test multiple WITH queries -- --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE --- y (id) AS (VALUES (1)), --- x (id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5) ---SELECT * FROM x; +WITH RECURSIVE + y (id) AS (VALUES (1)), + x (id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5) +SELECT * FROM x; -- forward reference OK --- [SPARK-24497] Support recursive SQL query +-- [NOTE] Spark SQL doesn't support forward references --WITH RECURSIVE -- x(id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5), -- y(id) AS (values (1)) -- SELECT * FROM x; --- [SPARK-24497] Support recursive SQL query +-- [SPARK-28405] Join with USING caluse doesn't hide original tables --WITH RECURSIVE -- x(id) AS -- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), @@ -411,7 +489,7 @@ insert into graph values -- (VALUES (1) UNION ALL SELECT id+1 FROM y WHERE id < 10) -- SELECT y.*, x.* FROM y LEFT JOIN x USING (id); --- [SPARK-24497] Support recursive SQL query +-- [SPARK-28405] Join with USING caluse doesn't hide original tables --WITH RECURSIVE -- x(id) AS -- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), @@ -419,25 +497,23 @@ insert into graph values -- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 10) -- SELECT y.*, x.* FROM y LEFT JOIN x USING (id); --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE --- x(id) AS --- (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), --- y(id) AS --- (SELECT * FROM x UNION ALL SELECT * FROM x), --- z(id) AS --- (SELECT * FROM x UNION ALL SELECT id+1 FROM z WHERE id < 10) --- SELECT * FROM z; - --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE --- x(id) AS --- (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), --- y(id) AS --- (SELECT * FROM x UNION ALL SELECT * FROM x), --- z(id) AS --- (SELECT * FROM y UNION ALL SELECT id+1 FROM z WHERE id < 10) --- SELECT * FROM z; +WITH RECURSIVE + x(id) AS + (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), + y(id) AS + (SELECT * FROM x UNION ALL SELECT * FROM x), + z(id) AS + (SELECT * FROM x UNION ALL SELECT id+1 FROM z WHERE id < 10) + SELECT * FROM z; + +WITH RECURSIVE + x(id) AS + (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), + y(id) AS + (SELECT * FROM x UNION ALL SELECT * FROM x), + z(id) AS + (SELECT * FROM y UNION ALL SELECT id+1 FROM z WHERE id < 10) + SELECT * FROM z; -- -- Test WITH attached to a data-modifying statement @@ -484,32 +560,26 @@ DROP TABLE y; -- -- INTERSECT --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x) + SELECT * FROM x; --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x) + SELECT * FROM x; -- EXCEPT --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x) + SELECT * FROM x; --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x) + SELECT * FROM x; -- no non-recursive term --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT n FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT n FROM x) + SELECT * FROM x; -- recursive term in the left hand side (strictly speaking, should allow this) --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1) + SELECT * FROM x; -- [ORIGINAL SQL] --CREATE TEMPORARY TABLE y (a INTEGER); @@ -520,123 +590,119 @@ INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)); -- LEFT JOIN --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 --- UNION ALL --- SELECT x.n+1 FROM y LEFT JOIN x ON x.n = y.a WHERE n < 10) ---SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 + UNION ALL + SELECT x.n+1 FROM y LEFT JOIN x ON x.n = y.a WHERE n < 10) +SELECT * FROM x; -- RIGHT JOIN --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 --- UNION ALL --- SELECT x.n+1 FROM x RIGHT JOIN y ON x.n = y.a WHERE n < 10) ---SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 + UNION ALL + SELECT x.n+1 FROM x RIGHT JOIN y ON x.n = y.a WHERE n < 10) +SELECT * FROM x; -- FULL JOIN --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 --- UNION ALL --- SELECT x.n+1 FROM x FULL JOIN y ON x.n = y.a WHERE n < 10) ---SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 + UNION ALL + SELECT x.n+1 FROM x FULL JOIN y ON x.n = y.a WHERE n < 10) +SELECT * FROM x; -- subquery --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x --- WHERE n IN (SELECT * FROM x)) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x + WHERE n IN (SELECT * FROM x)) + SELECT * FROM x; -- aggregate functions --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT count(*) FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT count(*) FROM x) + SELECT * FROM x; --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT sum(n) FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT sum(n) FROM x) + SELECT * FROM x; -- ORDER BY --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x ORDER BY 1) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x ORDER BY 1) + SELECT * FROM x; -- LIMIT/OFFSET --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x LIMIT 10 OFFSET 1) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x LIMIT 10 OFFSET 1) + SELECT * FROM x; -- FOR UPDATE --- [SPARK-24497] Support recursive SQL query +-- [NOTE] Spark SQL doesn't support FOR UPDATE --WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x FOR UPDATE) -- SELECT * FROM x; -- target list has a recursive query name --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(id) AS (values (1) --- UNION ALL --- SELECT (SELECT * FROM x) FROM x WHERE id < 5 ---) SELECT * FROM x; +WITH RECURSIVE x(id) AS (values (1) + UNION ALL + SELECT (SELECT * FROM x) FROM x WHERE id < 5 +) SELECT * FROM x; -- mutual recursive query (not implemented) --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE --- x (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM y WHERE id < 5), --- y (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 5) ---SELECT * FROM x; +WITH RECURSIVE + x (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM y WHERE id < 5), + y (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 5) +SELECT * FROM x; -- non-linear recursion is not allowed --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE foo(i) AS --- (values (1) --- UNION ALL --- (SELECT i+1 FROM foo WHERE i < 10 --- UNION ALL --- SELECT i+1 FROM foo WHERE i < 5) ---) SELECT * FROM foo; - --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE foo(i) AS --- (values (1) --- UNION ALL --- SELECT * FROM --- (SELECT i+1 FROM foo WHERE i < 10 --- UNION ALL --- SELECT i+1 FROM foo WHERE i < 5) AS t ---) SELECT * FROM foo; - --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE foo(i) AS --- (values (1) --- UNION ALL --- (SELECT i+1 FROM foo WHERE i < 10 --- EXCEPT --- SELECT i+1 FROM foo WHERE i < 5) ---) SELECT * FROM foo; - --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE foo(i) AS --- (values (1) --- UNION ALL --- (SELECT i+1 FROM foo WHERE i < 10 --- INTERSECT --- SELECT i+1 FROM foo WHERE i < 5) ---) SELECT * FROM foo; +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + UNION ALL + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo; + +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + SELECT * FROM + (SELECT i+1 FROM foo WHERE i < 10 + UNION ALL + SELECT i+1 FROM foo WHERE i < 5) AS t +) SELECT * FROM foo; + +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + EXCEPT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo; + +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + INTERSECT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo; -- Wrong type induced from non-recursive term --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --WITH RECURSIVE foo(i) AS -- (SELECT i FROM (VALUES(1),(2)) t(i) -- UNION ALL -- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) --SELECT * FROM foo; +WITH RECURSIVE foo(i) AS + (SELECT i FROM (VALUES(1),(2)) t(i) + UNION ALL + SELECT cast((i+1) AS decimal(10,0)) FROM foo WHERE i < 10) +SELECT * FROM foo; -- rejects different typmod, too (should we allow this?) --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --WITH RECURSIVE foo(i) AS -- (SELECT i::numeric(3,0) FROM (VALUES(1),(2)) t(i) -- UNION ALL -- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) --SELECT * FROM foo; +WITH RECURSIVE foo(i) AS + (SELECT cast(i as decimal(3,0)) FROM (VALUES(1),(2)) t(i) + UNION ALL + SELECT cast((i+1) AS decimal(10,0)) FROM foo WHERE i < 10) +SELECT * FROM foo; -- [NOTE] Spark SQL doesn't support RULEs -- disallow OLD/NEW reference in CTE @@ -668,18 +734,17 @@ with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q; -- -- test for nested-recursive-WITH bug -- --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE t(j) AS ( --- WITH RECURSIVE s(i) AS ( --- VALUES (1) --- UNION ALL --- SELECT i+1 FROM s WHERE i < 10 --- ) --- SELECT i FROM s --- UNION ALL --- SELECT j+1 FROM t WHERE j < 10 ---) ---SELECT * FROM t; +WITH RECURSIVE t(j) AS ( + WITH RECURSIVE s(i) AS ( + VALUES (1) + UNION ALL + SELECT i+1 FROM s WHERE i < 10 + ) + SELECT i FROM s + UNION ALL + SELECT j+1 FROM t WHERE j < 10 +) +SELECT * FROM t; -- -- test WITH attached to intermediate-level set operation @@ -701,22 +766,20 @@ WITH outermost(x) AS ( ) SELECT * FROM outermost ORDER BY 1; --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE outermost(x) AS ( --- SELECT 1 --- UNION (WITH innermost as (SELECT 2) --- SELECT * FROM outermost --- UNION SELECT * FROM innermost) ---) ---SELECT * FROM outermost ORDER BY 1; +WITH RECURSIVE outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1; --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE outermost(x) AS ( --- WITH innermost as (SELECT 2 FROM outermost) -- fail --- SELECT * FROM innermost --- UNION SELECT * from outermost ---) ---SELECT * FROM outermost ORDER BY 1; +WITH RECURSIVE outermost(x) AS ( + WITH innermost as (SELECT 2 FROM outermost) -- fail + SELECT * FROM innermost + UNION SELECT * from outermost +) +SELECT * FROM outermost ORDER BY 1; -- -- This test will fail with the old implementation of PARAM_EXEC parameter @@ -737,8 +800,7 @@ SELECT * FROM outermost ORDER BY 1; -- Test CTEs read in non-initialization orders -- --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE +-- [NOTE] Spark SQL doesn't support recursive reference in an inner CTE -- tab(id_key,link) AS (VALUES (1,17), (2,17), (3,17), (4,17), (6,17), (5,17)), -- iter (id_key, row_type, link) AS ( -- SELECT 0, 'base', 17 @@ -764,7 +826,7 @@ SELECT * FROM outermost ORDER BY 1; -- ) --SELECT * FROM iter; --- [SPARK-24497] Support recursive SQL query +-- [NOTE] Spark SQL doesn't support recursive reference in an inner CTE --WITH RECURSIVE -- tab(id_key,link) AS (VALUES (1,17), (2,17), (3,17), (4,17), (6,17), (5,17)), -- iter (id_key, row_type, link) AS ( @@ -796,7 +858,7 @@ SELECT * FROM outermost ORDER BY 1; -- -- INSERT ... RETURNING --- [SPARK-28147] Support RETURNING clause +-- [NOTE] Spark SQL doesn't support RETURNING --WITH t AS ( -- INSERT INTO y -- VALUES @@ -895,7 +957,7 @@ SELECT * FROM outermost ORDER BY 1; --SELECT * FROM bug6051_2; -- a truly recursive CTE in the same list --- [SPARK-24497] Support recursive SQL query +-- [NOTE] Spark SQL doesn't support RETURNING --WITH RECURSIVE t(a) AS ( -- SELECT 0 -- UNION ALL @@ -1003,7 +1065,6 @@ INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)); --CREATE TEMPORARY TABLE yy (a INTEGER); CREATE TABLE yy (a INTEGER) USING parquet; --- [SPARK-24497] Support recursive SQL query -- [SPARK-28147] Support RETURNING clause --WITH RECURSIVE t1 AS ( -- INSERT INTO y SELECT * FROM y RETURNING * @@ -1015,7 +1076,6 @@ CREATE TABLE yy (a INTEGER) USING parquet; SELECT * FROM y; SELECT * FROM yy; --- [SPARK-24497] Support recursive SQL query -- [SPARK-28147] Support RETURNING clause --WITH RECURSIVE t1 AS ( -- INSERT INTO yy SELECT * FROM t2 RETURNING * @@ -1148,7 +1208,7 @@ SELECT * FROM parent; -- error cases -- data-modifying WITH tries to use its own output --- [SPARK-24497] Support recursive SQL query +-- [NOTE] Spark SQL doesn't support INSERT in WITH --WITH RECURSIVE t AS ( -- INSERT INTO y -- SELECT * FROM t @@ -1156,7 +1216,7 @@ SELECT * FROM parent; --VALUES(FALSE); -- no RETURNING in a referenced data-modifying WITH --- [SPARK-24497] Support recursive SQL query +-- [NOTE] Spark SQL doesn't support INSERT in WITH --WITH t AS ( -- INSERT INTO y VALUES(0) --) @@ -1199,6 +1259,9 @@ drop table test; -- -- Clean up -- +DROP VIEW nums; +DROP VIEW vsubdepartment; +DROP VIEW sums_1_100; DROP TABLE department; DROP TABLE tree; diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out index 10fcee1469398..03896e3ffa9c3 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out @@ -227,6 +227,7 @@ REAL false RECORDREADER false RECORDWRITER false RECOVER false +RECURSIVE true REDUCE false REFERENCES true REFRESH false @@ -397,6 +398,7 @@ OVERLAPS PERCENTILE_CONT PERCENTILE_DISC PRIMARY +RECURSIVE REFERENCES RIGHT SELECT 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 new file mode 100644 index 0000000000000..8b8bdeeac3a0e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out @@ -0,0 +1,965 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +WITH r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`r`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 67, + "stopIndex" : 67, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r AS ( + SELECT 0 AS level + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +WITH RECURSIVE r(c) AS ( + SELECT 'a' + UNION ALL + SELECT c || char(ascii(substr(c, -1)) + 1) FROM r WHERE LENGTH(c) < 10 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +a +ab +abc +abcd +abcde +abcdef +abcdefg +abcdefgh +abcdefghi +abcdefghij + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cteRecursionLevelLimit + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r LIMIT 10 +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION + SELECT (level + 1) % 10 FROM r +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT level, level FROM r ORDER BY 1 LIMIT 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cteRecursionLevelLimit + + +-- !query +WITH RECURSIVE r(level) AS ( + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "WITH RECURSIVE r(level) AS (\n SELECT level + 1 FROM r WHERE level < 9\n)\nSELECT * FROM r" + } ] +} + + +-- !query +WITH RECURSIVE r(level) AS ( + SELECT level + 1 FROM r WHERE level < 9 + UNION ALL + VALUES 0 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`level`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 66, + "fragment" : "level" + } ] +} + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + INTERSECT + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 111, + "fragment" : "WITH RECURSIVE r(level) AS (\n VALUES 0\n INTERSECT\n SELECT level + 1 FROM r WHERE level < 9\n)\nSELECT * FROM r" + } ] +} + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE (SELECT SUM(level) FROM r) < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`r`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 109, + "stopIndex" : 109, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE + t1 AS ( + SELECT 1 AS level + UNION ( + WITH t2 AS (SELECT level + 1 FROM t1 WHERE level < 10) + SELECT * FROM t2 + ) + ) +SELECT * FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 100, + "stopIndex" : 101, + "fragment" : "t1" + } ] +} + + +-- !query +SET spark.sql.legacy.ctePrecedencePolicy=CORRECTED +-- !query schema +struct +-- !query output +spark.sql.legacy.ctePrecedencePolicy CORRECTED + + +-- !query +WITH + t1 AS (SELECT 1), + t2 AS ( + WITH RECURSIVE + t1 AS ( + SELECT 1 AS level + UNION ( + WITH t3 AS (SELECT level + 1 FROM t1 WHERE level < 10) + SELECT * FROM t3 + ) + ) + SELECT * FROM t1 + ) +SELECT * FROM t2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 155, + "stopIndex" : 156, + "fragment" : "t1" + } ] +} + + +-- !query +SET spark.sql.legacy.ctePrecedencePolicy=EXCEPTION +-- !query schema +struct +-- !query output +spark.sql.legacy.ctePrecedencePolicy EXCEPTION + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT r1.level + 1, r1.data + FROM r AS r1 + JOIN r AS r2 ON r2.data = r1.data + WHERE r1.level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836" +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + LEFT OUTER JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM ( + SELECT 0 AS data + ) AS t + LEFT OUTER JOIN r ON r.data = t.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 148, + "stopIndex" : 148, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM ( + SELECT 0 AS data + ) AS t + RIGHT OUTER JOIN r ON r.data = t.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + RIGHT OUTER JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 98, + "stopIndex" : 98, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + FULL OUTER JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 98, + "stopIndex" : 98, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + LEFT SEMI JOIN ( + SELECT 0 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, data + FROM ( + SELECT 0 AS level, 0 AS data + ) AS t + LEFT SEMI JOIN r ON r.data = t.data AND r.level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 157, + "stopIndex" : 157, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, r.data + FROM r + LEFT ANTI JOIN ( + SELECT -1 AS data + ) AS t ON t.data = r.data + WHERE r.level < 9 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, data + FROM ( + SELECT 0 AS level, 0 AS data + ) AS t + LEFT ANTI JOIN r ON r.data = t.data AND r.level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 157, + "stopIndex" : 157, + "fragment" : "r" + } ] +} + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 1L) + UNION ALL + SELECT MAX(level) + 1, SUM(data) FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 105, + "stopIndex" : 105, + "fragment" : "r" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW rv AS +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM rv +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +DROP VIEW rv +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE rt(level INT) USING csv +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +INSERT INTO rt SELECT * FROM r +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * from rt +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +DROP TABLE rt +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE rt2(level INT) USING csv +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +FROM r +INSERT INTO rt2 SELECT * +INSERT INTO rt2 SELECT * +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "RECURSIVE_CTE_WHEN_INLINING_IS_FORCED", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 160, + "fragment" : "WITH RECURSIVE r(level) AS (\n VALUES (0)\n UNION ALL\n SELECT level + 1 FROM r WHERE level < 9\n)\nFROM r\nINSERT INTO rt2 SELECT *\nINSERT INTO rt2 SELECT *" + } ] +} + + +-- !query +DROP TABLE rt2 +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH RECURSIVE + r1 AS ( + SELECT 0 AS level + UNION ALL + SELECT level + 1 FROM r1 WHERE level < 9 + ), + r2 AS ( + SELECT 10 AS level + UNION ALL + SELECT level + 1 FROM r2 WHERE level < 19 + ) +SELECT * +FROM r1 +JOIN r2 ON r2.level = r1.level + 10 +-- !query schema +struct +-- !query output +0 10 +1 11 +2 12 +3 13 +4 14 +5 15 +6 16 +7 17 +8 18 +9 19 + + +-- !query +WITH RECURSIVE r AS ( + SELECT 0 AS level + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 +) +SELECT * +FROM r AS r1 +JOIN r AS r2 ON r2.level = r1.level +-- !query schema +struct +-- !query output +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 + + +-- !query +WITH RECURSIVE r2 AS ( + WITH RECURSIVE r1 AS ( + SELECT 0 AS innerlevel + UNION ALL + SELECT innerlevel + 1 FROM r1 WHERE innerlevel < 3 + ) + SELECT 0 AS outerlevel, innerlevel FROM r1 + UNION ALL + SELECT outerlevel + 1, innerlevel FROM r2 WHERE outerlevel < 3 +) +SELECT * FROM r2 +-- !query schema +struct +-- !query output +0 0 +0 1 +0 2 +0 3 +1 0 +1 1 +1 2 +1 3 +2 0 +2 1 +2 2 +2 3 +3 0 +3 1 +3 2 +3 3 + + +-- !query +WITH RECURSIVE r(level) AS ( + WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 + ) + SELECT * FROM r + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 +1 +1 +2 +2 +3 +3 + + +-- !query +WITH RECURSIVE r(level) AS ( + (WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 + ) + SELECT * FROM r) + UNION ALL + SELECT level + 1 FROM r WHERE level < 3 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 +1 +1 +2 +2 +2 +3 +3 +3 +3 + + +-- !query +CREATE TEMPORARY VIEW routes(origin, destination) AS VALUES + ('New York', 'Washington'), + ('New York', 'Boston'), + ('Boston', 'New York'), + ('Washington', 'Boston'), + ('Washington', 'Raleigh') +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH RECURSIVE destinations_from_new_york AS ( + SELECT 'New York' AS destination, ARRAY('New York') AS path, 0 AS length + UNION ALL + SELECT r.destination, CONCAT(d.path, ARRAY(r.destination)), d.length + 1 + FROM routes AS r + JOIN destinations_from_new_york AS d ON d.destination = r.origin AND NOT ARRAY_CONTAINS(d.path, r.destination) +) +SELECT * FROM destinations_from_new_york +-- !query schema +struct,length:int> +-- !query output +Boston ["New York","Boston"] 1 +Boston ["New York","Washington","Boston"] 2 +New York ["New York"] 0 +Raleigh ["New York","Washington","Raleigh"] 2 +Washington ["New York","Washington"] 1 + + +-- !query +DROP VIEW routes +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH RECURSIVE fibonacci AS ( + VALUES (0, 1) AS t(a, b) + UNION ALL + SELECT b, a + b FROM fibonacci WHERE a < 10 +) +SELECT a FROM fibonacci ORDER BY a +-- !query schema +struct +-- !query output +0 +1 +1 +2 +3 +5 +8 +13 diff --git a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out index 97a3f6c8f2ee2..d8cdaf982dbb6 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out @@ -956,6 +956,50 @@ Output [4]: [key#x, max(val)#x, key#x, max(val)#x] Arguments: isFinalPlan=false +-- !query +EXPLAIN FORMATTED + WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 + ) +SELECT * FROM r +-- !query schema +struct +-- !query output +== Physical Plan == +* Project (6) ++- UnionLoop (1) + :- LocalRelation (2) + +- Project (5) + +- Filter (4) + +- UnionLoopRef (3) + + +(1) UnionLoop +Loop id: xxxx +Output [1]: [col1#x] +Limit: None + +(2) LocalRelation +Arguments: [col1#x] + +(3) UnionLoopRef +Loop id: xxxx +Output [1]: [level#x] +Accumulated: false + +(4) Filter +Arguments: (level#x < 9) + +(5) Project +Arguments: [(level#x + 1) AS (level + 1)#x] + +(6) Project [codegen id : 1] +Output [1]: [col1#x AS level#x] +Input [1]: [col1#x] + + -- !query EXPLAIN FORMATTED CREATE VIEW explain_view AS diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index e21d968eb252e..b6c69aae4a530 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -851,6 +851,50 @@ Join type: Inner Join condition: None +-- !query +EXPLAIN FORMATTED + WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r WHERE level < 9 + ) +SELECT * FROM r +-- !query schema +struct +-- !query output +== Physical Plan == +* Project (6) ++- UnionLoop (1) + :- LocalRelation (2) + +- Project (5) + +- Filter (4) + +- UnionLoopRef (3) + + +(1) UnionLoop +Loop id: xxxx +Output [1]: [col1#x] +Limit: None + +(2) LocalRelation +Arguments: [col1#x] + +(3) UnionLoopRef +Loop id: xxxx +Output [1]: [level#x] +Accumulated: false + +(4) Filter +Arguments: (level#x < 9) + +(5) Project +Arguments: [(level#x + 1) AS (level + 1)#x] + +(6) Project [codegen id : 1] +Output [1]: [col1#x AS level#x] +Input [1]: [col1#x] + + -- !query EXPLAIN FORMATTED CREATE VIEW explain_view AS diff --git a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out index be2303a716da5..1c0219525a7ed 100644 --- a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out @@ -227,6 +227,7 @@ REAL false RECORDREADER false RECORDWRITER false RECOVER false +RECURSIVE false REDUCE false REFERENCES false REFRESH false diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out index af1c33023a912..2b5b6f90e197a 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out @@ -21,6 +21,199 @@ struct 5 +-- !query +WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t WHERE n < 100 +) +SELECT sum(n) FROM t +-- !query schema +struct +-- !query output +5050 + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT (VALUES(1)) +UNION ALL + SELECT n+1 FROM t WHERE n < 5 +) +SELECT * FROM t +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 + + +-- !query +CREATE TEMPORARY VIEW nums AS +WITH RECURSIVE nums (n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM nums WHERE n < 5 +) +SELECT * FROM nums +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM nums +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW nums AS +WITH RECURSIVE nums (n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM nums WHERE n < 6 +) +SELECT * FROM nums +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM nums +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 +6 + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT 1 +UNION + SELECT 10-n FROM t) +SELECT * FROM t +-- !query schema +struct +-- !query output +1 +9 + + +-- !query +WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t) +SELECT * FROM t LIMIT 10 +-- !query schema +struct +-- !query output +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT 1 +UNION + SELECT n+1 FROM t) +SELECT * FROM t LIMIT 10 +-- !query schema +struct +-- !query output +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +WITH q AS (SELECT 'foo' AS x) +SELECT x FROM q +-- !query schema +struct +-- !query output +foo + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT 'foo' +UNION ALL + SELECT n || ' bar' FROM t WHERE length(n) < 20 +) +SELECT n AS is_text FROM t +-- !query schema +struct +-- !query output +foo +foo bar +foo bar bar +foo bar bar bar +foo bar bar bar bar +foo bar bar bar bar bar + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT '7' +UNION ALL + SELECT n+1 FROM t WHERE n < 10 +) +SELECT n FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.DATA_TYPE", + "sqlState" : "42836", + "messageParameters" : { + "fromDataType" : "\"STRING\"", + "toDataType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 71, + "fragment" : "t" + } ] +} + + -- !query CREATE TABLE department ( id INTEGER, -- department ID @@ -30,19 +223,733 @@ CREATE TABLE department ( -- !query schema struct<> -- !query output - + + + +-- !query +INSERT INTO department VALUES (0, NULL, 'ROOT') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (1, 0, 'A') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (2, 1, 'B') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (3, 2, 'C') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (4, 2, 'D') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (5, 0, 'E') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (6, 4, 'F') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (7, 5, 'G') +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH RECURSIVE subdepartment AS +( + SELECT name as root_name, * FROM department WHERE name = 'A' + + UNION ALL + + SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id +) +SELECT * FROM subdepartment ORDER BY name +-- !query schema +struct +-- !query output +A 1 0 A +A 2 1 B +A 3 2 C +A 4 2 D +A 6 4 F + + +-- !query +WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +( + SELECT 1, * FROM department WHERE name = 'A' + + UNION ALL + + SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id +) +SELECT * FROM subdepartment ORDER BY name +-- !query schema +struct +-- !query output +1 1 0 A +2 2 1 B +3 3 2 C +3 4 2 D +4 6 4 F + + +-- !query +WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +( + SELECT 1, * FROM department WHERE name = 'A' + + UNION ALL + + SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id +) +SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name +-- !query schema +struct +-- !query output +2 2 1 B +3 3 2 C +3 4 2 D +4 6 4 F + + +-- !query +WITH RECURSIVE subdepartment AS +( + SELECT * FROM department WHERE name = 'A' +) +SELECT * FROM subdepartment ORDER BY name +-- !query schema +struct +-- !query output +1 0 A + + +-- !query +SET spark.sql.cteRecursionLevelLimit=200 +-- !query schema +struct +-- !query output +spark.sql.cteRecursionLevelLimit 200 + + +-- !query +SELECT count(*) FROM ( + WITH RECURSIVE t(n) AS ( + SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 200 + ) + SELECT * FROM t) AS t WHERE n < ( + SELECT count(*) FROM ( + WITH RECURSIVE t(n) AS ( + SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 100 + ) + SELECT * FROM t WHERE n < 50000 + ) AS t WHERE n < 100) +-- !query schema +struct +-- !query output +98 + + +-- !query +SET spark.sql.cteRecursionLevelLimit=100 +-- !query schema +struct +-- !query output +spark.sql.cteRecursionLevelLimit 100 + + +-- !query +WITH q1(x,y) AS ( + SELECT hundred, sum(ten) FROM tenk1 GROUP BY hundred + ) +SELECT count(*) FROM q1 WHERE y > (SELECT sum(y)/100 FROM q1 qsub) +-- !query schema +struct +-- !query output +50 + + +-- !query +CREATE TEMPORARY VIEW vsubdepartment AS + WITH RECURSIVE subdepartment AS + ( + SELECT * FROM department WHERE name = 'A' + UNION ALL + SELECT d.* FROM department AS d, subdepartment AS sd + WHERE d.parent_department = sd.id + ) + SELECT * FROM subdepartment +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM vsubdepartment ORDER BY name +-- !query schema +struct +-- !query output +1 0 A +2 1 B +3 2 C +4 2 D +6 4 F + + +-- !query +CREATE VIEW sums_1_100 AS +WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t WHERE n < 100 +) +SELECT sum(n) AS sum FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM sums_1_100 +-- !query schema +struct +-- !query output +5050 + + +-- !query +WITH RECURSIVE t(i,j) AS ( + VALUES (1,2) + UNION ALL + SELECT t2.i, t.j+1 FROM + (SELECT 2 AS i UNION ALL SELECT 3 AS i) AS t2 + JOIN t ON (t2.i = t.i+1)) + + SELECT * FROM t +-- !query schema +struct +-- !query output +1 2 +2 3 +3 4 + + +-- !query +CREATE TABLE tree( + id INTEGER, + parent_id INTEGER +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO tree +VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), + (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11) +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH RECURSIVE t(id, path) AS ( + VALUES(1,cast(array() as array)) +UNION ALL + SELECT tree.id, t.path || array(tree.id) + FROM tree JOIN t ON (tree.parent_id = t.id) +) +SELECT t1.*, t2.* FROM t AS t1 JOIN t AS t2 ON + (t1.path[0] = t2.path[0] AND + size(t1.path) = 1 AND + size(t2.path) > 1) + ORDER BY t1.id, t2.id +-- !query schema +struct,id:int,path:array> +-- !query output +2 [2] 4 [2,4] +2 [2] 5 [2,5] +2 [2] 6 [2,6] +2 [2] 9 [2,4,9] +2 [2] 10 [2,4,10] +2 [2] 14 [2,4,9,14] +3 [3] 7 [3,7] +3 [3] 8 [3,8] +3 [3] 11 [3,7,11] +3 [3] 12 [3,7,12] +3 [3] 13 [3,7,13] +3 [3] 15 [3,7,11,15] +3 [3] 16 [3,7,11,16] + + +-- !query +WITH RECURSIVE t(id, path) AS ( + VALUES(1,cast(array() as array)) +UNION ALL + SELECT tree.id, t.path || array(tree.id) + FROM tree JOIN t ON (tree.parent_id = t.id) +) +SELECT t1.id, count(*) FROM t AS t1 JOIN t AS t2 ON + (t1.path[0] = t2.path[0] AND + size(t1.path) = 1 AND + size(t2.path) > 1) + GROUP BY t1.id + ORDER BY t1.id +-- !query schema +struct +-- !query output +2 6 +3 7 + + +-- !query +WITH RECURSIVE t(id, path) AS ( + VALUES(1,cast(array() as array)) +UNION ALL + SELECT tree.id, t.path || array(tree.id) + FROM tree JOIN t ON (tree.parent_id = t.id) +) +SELECT t1.id, t2.path, struct(t2.*) FROM t AS t1 JOIN t AS t2 ON +(t1.id=t2.id) +-- !query schema +struct,struct(id, path):struct>> +-- !query output +1 [] {"id":1,"path":[]} +10 [2,4,10] {"id":10,"path":[2,4,10]} +11 [3,7,11] {"id":11,"path":[3,7,11]} +12 [3,7,12] {"id":12,"path":[3,7,12]} +13 [3,7,13] {"id":13,"path":[3,7,13]} +14 [2,4,9,14] {"id":14,"path":[2,4,9,14]} +15 [3,7,11,15] {"id":15,"path":[3,7,11,15]} +16 [3,7,11,16] {"id":16,"path":[3,7,11,16]} +2 [2] {"id":2,"path":[2]} +3 [3] {"id":3,"path":[3]} +4 [2,4] {"id":4,"path":[2,4]} +5 [2,5] {"id":5,"path":[2,5]} +6 [2,6] {"id":6,"path":[2,6]} +7 [3,7] {"id":7,"path":[3,7]} +8 [3,8] {"id":8,"path":[3,8]} +9 [2,4,9] {"id":9,"path":[2,4,9]} + + +-- !query +create table graph( f int, t int, label string ) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into graph values + (1, 2, 'arc 1 -> 2'), + (1, 3, 'arc 1 -> 3'), + (2, 3, 'arc 2 -> 3'), + (1, 4, 'arc 1 -> 4'), + (4, 5, 'arc 4 -> 5'), + (5, 1, 'arc 5 -> 1') +-- !query schema +struct<> +-- !query output + + + +-- !query +with recursive search_graph(f, t, label, path, cycle) as ( + select *, array(struct(g.f, g.t)), false from graph g + union all + select g.*, path || array(struct(g.f, g.t)), array_contains(path, struct(g.f, g.t)) + from graph g, search_graph sg + where g.f = sg.t and not cycle +) +select * from search_graph +-- !query schema +struct>,cycle:boolean> +-- !query output +1 2 arc 1 -> 2 [{"f":1,"t":2}] false +1 2 arc 1 -> 2 [{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":2}] false +1 2 arc 1 -> 2 [{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":2}] false +1 2 arc 1 -> 2 [{"f":5,"t":1},{"f":1,"t":2}] false +1 3 arc 1 -> 3 [{"f":1,"t":3}] false +1 3 arc 1 -> 3 [{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":3}] false +1 3 arc 1 -> 3 [{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":3}] false +1 3 arc 1 -> 3 [{"f":5,"t":1},{"f":1,"t":3}] false +1 4 arc 1 -> 4 [{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":4}] true +1 4 arc 1 -> 4 [{"f":1,"t":4}] false +1 4 arc 1 -> 4 [{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":4}] false +1 4 arc 1 -> 4 [{"f":5,"t":1},{"f":1,"t":4}] false +2 3 arc 2 -> 3 [{"f":1,"t":2},{"f":2,"t":3}] false +2 3 arc 2 -> 3 [{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":2},{"f":2,"t":3}] false +2 3 arc 2 -> 3 [{"f":2,"t":3}] false +2 3 arc 2 -> 3 [{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":2},{"f":2,"t":3}] false +2 3 arc 2 -> 3 [{"f":5,"t":1},{"f":1,"t":2},{"f":2,"t":3}] false +4 5 arc 4 -> 5 [{"f":1,"t":4},{"f":4,"t":5}] false +4 5 arc 4 -> 5 [{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":4},{"f":4,"t":5}] true +4 5 arc 4 -> 5 [{"f":4,"t":5}] false +4 5 arc 4 -> 5 [{"f":5,"t":1},{"f":1,"t":4},{"f":4,"t":5}] false +5 1 arc 5 -> 1 [{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1}] false +5 1 arc 5 -> 1 [{"f":4,"t":5},{"f":5,"t":1}] false +5 1 arc 5 -> 1 [{"f":5,"t":1},{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1}] true +5 1 arc 5 -> 1 [{"f":5,"t":1}] false + + +-- !query +with recursive search_graph(f, t, label, path, cycle) as ( + select *, array(struct(g.f, g.t)), false from graph g + union all + select g.*, path || array(struct(g.f, g.t)), array_contains(path, struct(g.f, g.t)) + from graph g, search_graph sg + where g.f = sg.t and not cycle +) +select * from search_graph order by path +-- !query schema +struct>,cycle:boolean> +-- !query output +1 2 arc 1 -> 2 [{"f":1,"t":2}] false +2 3 arc 2 -> 3 [{"f":1,"t":2},{"f":2,"t":3}] false +1 3 arc 1 -> 3 [{"f":1,"t":3}] false +1 4 arc 1 -> 4 [{"f":1,"t":4}] false +4 5 arc 4 -> 5 [{"f":1,"t":4},{"f":4,"t":5}] false +5 1 arc 5 -> 1 [{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1}] false +1 2 arc 1 -> 2 [{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":2}] false +2 3 arc 2 -> 3 [{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":2},{"f":2,"t":3}] false +1 3 arc 1 -> 3 [{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":3}] false +1 4 arc 1 -> 4 [{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":4}] true +2 3 arc 2 -> 3 [{"f":2,"t":3}] false +4 5 arc 4 -> 5 [{"f":4,"t":5}] false +5 1 arc 5 -> 1 [{"f":4,"t":5},{"f":5,"t":1}] false +1 2 arc 1 -> 2 [{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":2}] false +2 3 arc 2 -> 3 [{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":2},{"f":2,"t":3}] false +1 3 arc 1 -> 3 [{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":3}] false +1 4 arc 1 -> 4 [{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":4}] false +4 5 arc 4 -> 5 [{"f":4,"t":5},{"f":5,"t":1},{"f":1,"t":4},{"f":4,"t":5}] true +5 1 arc 5 -> 1 [{"f":5,"t":1}] false +1 2 arc 1 -> 2 [{"f":5,"t":1},{"f":1,"t":2}] false +2 3 arc 2 -> 3 [{"f":5,"t":1},{"f":1,"t":2},{"f":2,"t":3}] false +1 3 arc 1 -> 3 [{"f":5,"t":1},{"f":1,"t":3}] false +1 4 arc 1 -> 4 [{"f":5,"t":1},{"f":1,"t":4}] false +4 5 arc 4 -> 5 [{"f":5,"t":1},{"f":1,"t":4},{"f":4,"t":5}] false +5 1 arc 5 -> 1 [{"f":5,"t":1},{"f":1,"t":4},{"f":4,"t":5},{"f":5,"t":1}] true + + +-- !query +WITH RECURSIVE + y (id) AS (VALUES (1)), + x (id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5) +SELECT * FROM x +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 + + +-- !query +WITH RECURSIVE + x(id) AS + (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), + y(id) AS + (SELECT * FROM x UNION ALL SELECT * FROM x), + z(id) AS + (SELECT * FROM x UNION ALL SELECT id+1 FROM z WHERE id < 10) + SELECT * FROM z +-- !query schema +struct +-- !query output +1 +10 +10 +10 +2 +2 +3 +3 +3 +4 +4 +4 +5 +5 +5 +6 +6 +6 +7 +7 +7 +8 +8 +8 +9 +9 +9 + + +-- !query +WITH RECURSIVE + x(id) AS + (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), + y(id) AS + (SELECT * FROM x UNION ALL SELECT * FROM x), + z(id) AS + (SELECT * FROM y UNION ALL SELECT id+1 FROM z WHERE id < 10) + SELECT * FROM z +-- !query schema +struct +-- !query output +1 +1 +10 +10 +10 +10 +10 +10 +2 +2 +2 +2 +3 +3 +3 +3 +3 +3 +4 +4 +4 +4 +4 +4 +5 +5 +5 +5 +5 +5 +6 +6 +6 +6 +6 +6 +7 +7 +7 +7 +7 +7 +8 +8 +8 +8 +8 +8 +9 +9 +9 +9 +9 +9 + + +-- !query +CREATE TABLE y (a INTEGER) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE y +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x) + SELECT * FROM x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x)\n\tSELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x) + SELECT * FROM x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x)\n\tSELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x) + SELECT * FROM x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 75, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x)\n\tSELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x) + SELECT * FROM x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x)\n\tSELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT n FROM x) + SELECT * FROM x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 57, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT n FROM x)\n\tSELECT * FROM x" + } ] +} -- !query -INSERT INTO department VALUES (0, NULL, 'ROOT') +WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1) + SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`n`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "n" + } ] +} -- !query -INSERT INTO department VALUES (1, 0, 'A') +CREATE TABLE y (a INTEGER) USING parquet -- !query schema struct<> -- !query output @@ -50,7 +957,7 @@ struct<> -- !query -INSERT INTO department VALUES (2, 1, 'B') +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) -- !query schema struct<> -- !query output @@ -58,127 +965,352 @@ struct<> -- !query -INSERT INTO department VALUES (3, 2, 'C') +WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 + UNION ALL + SELECT x.n+1 FROM y LEFT JOIN x ON x.n = y.a WHERE n < 10) +SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 95, + "stopIndex" : 95, + "fragment" : "x" + } ] +} -- !query -INSERT INTO department VALUES (4, 2, 'D') +WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 + UNION ALL + SELECT x.n+1 FROM x RIGHT JOIN y ON x.n = y.a WHERE n < 10) +SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 83, + "stopIndex" : 83, + "fragment" : "x" + } ] +} -- !query -INSERT INTO department VALUES (5, 0, 'E') +WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 + UNION ALL + SELECT x.n+1 FROM x FULL JOIN y ON x.n = y.a WHERE n < 10) +SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 83, + "stopIndex" : 83, + "fragment" : "x" + } ] +} -- !query -INSERT INTO department VALUES (6, 4, 'F') +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x + WHERE n IN (SELECT * FROM x)) + SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 114, + "stopIndex" : 114, + "fragment" : "x" + } ] +} -- !query -INSERT INTO department VALUES (7, 5, 'G') +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT count(*) FROM x) + SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 65, + "stopIndex" : 65, + "fragment" : "x" + } ] +} -- !query -CREATE TABLE tree( - id INTEGER, - parent_id INTEGER -) USING parquet +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT sum(n) FROM x) + SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 63, + "stopIndex" : 63, + "fragment" : "x" + } ] +} -- !query -INSERT INTO tree -VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), - (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11) +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x ORDER BY 1) + SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 90, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x ORDER BY 1)\n SELECT * FROM x" + } ] +} -- !query -create table graph( f int, t int, label string ) USING parquet +WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x LIMIT 10 OFFSET 1) + SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 97, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x LIMIT 10 OFFSET 1)\n SELECT * FROM x" + } ] +} -- !query -insert into graph values - (1, 2, 'arc 1 -> 2'), - (1, 3, 'arc 1 -> 3'), - (2, 3, 'arc 2 -> 3'), - (1, 4, 'arc 1 -> 4'), - (4, 5, 'arc 4 -> 5'), - (5, 1, 'arc 5 -> 1') +WITH RECURSIVE x(id) AS (values (1) + UNION ALL + SELECT (SELECT * FROM x) FROM x WHERE id < 5 +) SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 77, + "stopIndex" : 77, + "fragment" : "x" + } ] +} -- !query -CREATE TABLE y (a INTEGER) USING parquet +WITH RECURSIVE + x (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM y WHERE id < 5), + y (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 5) +SELECT * FROM x -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`id`", + "proposal" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 73, + "stopIndex" : 74, + "fragment" : "id" + } ] +} -- !query -INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + UNION ALL + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836" +} -- !query -DROP TABLE y +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + SELECT * FROM + (SELECT i+1 FROM foo WHERE i < 10 + UNION ALL + SELECT i+1 FROM foo WHERE i < 5) AS t +) SELECT * FROM foo -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836" +} -- !query -CREATE TABLE y (a INTEGER) USING parquet +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + EXCEPT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo -- !query schema struct<> -- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836" +} +-- !query +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + INTERSECT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836" +} + -- !query -INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +WITH RECURSIVE foo(i) AS + (SELECT i FROM (VALUES(1),(2)) t(i) + UNION ALL + SELECT cast((i+1) AS decimal(10,0)) FROM foo WHERE i < 10) +SELECT * FROM foo -- !query schema struct<> -- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.DATA_TYPE", + "sqlState" : "42836", + "messageParameters" : { + "fromDataType" : "\"INT\"", + "toDataType" : "\"DECIMAL(10,0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 122, + "stopIndex" : 124, + "fragment" : "foo" + } ] +} +-- !query +WITH RECURSIVE foo(i) AS + (SELECT cast(i as decimal(3,0)) FROM (VALUES(1),(2)) t(i) + UNION ALL + SELECT cast((i+1) AS decimal(10,0)) FROM foo WHERE i < 10) +SELECT * FROM foo +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.DATA_TYPE", + "sqlState" : "42836", + "messageParameters" : { + "fromDataType" : "\"DECIMAL(3,0)\"", + "toDataType" : "\"DECIMAL(10,0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 144, + "stopIndex" : 146, + "fragment" : "foo" + } ] +} + -- !query with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q @@ -188,6 +1320,78 @@ struct 42 +-- !query +WITH RECURSIVE t(j) AS ( + WITH RECURSIVE s(i) AS ( + VALUES (1) + UNION ALL + SELECT i+1 FROM s WHERE i < 10 + ) + SELECT i FROM s + UNION ALL + SELECT j+1 FROM t WHERE j < 10 +) +SELECT * FROM t +-- !query schema +struct +-- !query output +1 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +2 +2 +3 +3 +3 +4 +4 +4 +4 +5 +5 +5 +5 +5 +6 +6 +6 +6 +6 +6 +7 +7 +7 +7 +7 +7 +7 +8 +8 +8 +8 +8 +8 +8 +8 +9 +9 +9 +9 +9 +9 +9 +9 +9 + + -- !query WITH outermost(x) AS ( SELECT 1 @@ -232,6 +1436,48 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +WITH RECURSIVE outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1 +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +WITH RECURSIVE outermost(x) AS ( + WITH innermost as (SELECT 2 FROM outermost) -- fail + SELECT * FROM innermost + UNION SELECT * from outermost +) +SELECT * FROM outermost ORDER BY 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`outermost`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 77, + "fragment" : "outermost" + } ] +} + + -- !query CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i AS string) || ' v' AS v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i) -- !query schema @@ -459,6 +1705,30 @@ struct<> +-- !query +DROP VIEW nums +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW vsubdepartment +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW sums_1_100 +-- !query schema +struct<> +-- !query output + + + -- !query DROP TABLE department -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala index d8956961440df..9b12ccc165564 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala @@ -53,6 +53,9 @@ trait SQLQueryTestHelper extends Logging { .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg") .replaceAll("CTERelationDef \\d+,", s"CTERelationDef xxxx,") .replaceAll("CTERelationRef \\d+,", s"CTERelationRef xxxx,") + .replaceAll("UnionLoop \\d+", "UnionLoop xxxx") + .replaceAll("UnionLoopRef \\d+,", "UnionLoopRef xxxx,") + .replaceAll("Loop id: \\d+", "Loop id: xxxx") .replaceAll("@\\w*,", s"@xxxxxxxx,") .replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 9bb35bb8719ea..224517085a9d7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -214,7 +214,7 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { val sessionHandle = client.openSession(user, "") val infoValue = client.getInfo(sessionHandle, GetInfoType.CLI_ODBC_KEYWORDS) // scalastyle:off line.size.limit - assert(infoValue.getStringValue == "ADD,AFTER,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,AT,AUTHORIZATION,BETWEEN,BIGINT,BINARY,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHAR,CHARACTER,CHECK,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPUTE,CONCATENATE,CONSTRAINT,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,DATA,DATABASE,DATABASES,DATE,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAY,DAYOFYEAR,DAYS,DBPROPERTIES,DEC,DECIMAL,DECLARE,DEFAULT,DEFINED,DELETE,DELIMITED,DESC,DESCRIBE,DFS,DIRECTORIES,DIRECTORY,DISTINCT,DISTRIBUTE,DIV,DOUBLE,DROP,ELSE,END,ESCAPE,ESCAPED,EXCEPT,EXCHANGE,EXCLUDE,EXISTS,EXPLAIN,EXPORT,EXTENDED,EXTERNAL,EXTRACT,FALSE,FETCH,FIELDS,FILEFORMAT,FILTER,FIRST,FLOAT,FOLLOWING,FOR,FOREIGN,FORMAT,FORMATTED,FROM,FULL,FUNCTION,FUNCTIONS,GENERATED,GLOBAL,GRANT,GROUP,GROUPING,HAVING,HOUR,HOURS,IDENTIFIER,IF,IGNORE,ILIKE,IMPORT,IN,INCLUDE,INDEX,INDEXES,INNER,INPATH,INPUTFORMAT,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,IS,ITEMS,JOIN,KEYS,LAST,LATERAL,LAZY,LEADING,LEFT,LIKE,LIMIT,LINES,LIST,LOAD,LOCAL,LOCATION,LOCK,LOCKS,LOGICAL,LONG,MACRO,MAP,MATCHED,MERGE,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTE,MINUTES,MONTH,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NATURAL,NO,NULL,NULLS,NUMERIC,OF,OFFSET,ON,ONLY,OPTION,OPTIONS,OR,ORDER,OUT,OUTER,OUTPUTFORMAT,OVER,OVERLAPS,OVERLAY,OVERWRITE,PARTITION,PARTITIONED,PARTITIONS,PERCENT,PERCENTILE_CONT,PERCENTILE_DISC,PIVOT,PLACING,POSITION,PRECEDING,PRIMARY,PRINCIPALS,PROPERTIES,PURGE,QUARTER,QUERY,RANGE,REAL,RECORDREADER,RECORDWRITER,RECOVER,REDUCE,REFERENCES,REFRESH,RENAME,REPAIR,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,REVOKE,RIGHT,ROLE,ROLES,ROLLBACK,ROLLUP,ROW,ROWS,SCHEMA,SCHEMAS,SECOND,SECONDS,SELECT,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SESSION_USER,SET,SETS,SHORT,SHOW,SINGLE,SKEWED,SMALLINT,SOME,SORT,SORTED,SOURCE,START,STATISTICS,STORED,STRATIFY,STRING,STRUCT,SUBSTR,SUBSTRING,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLE,TABLES,TABLESAMPLE,TARGET,TBLPROPERTIES,TERMINATED,THEN,TIME,TIMEDIFF,TIMESTAMP,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TO,TOUCH,TRAILING,TRANSACTION,TRANSACTIONS,TRANSFORM,TRIM,TRUE,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNION,UNIQUE,UNKNOWN,UNLOCK,UNPIVOT,UNSET,UPDATE,USE,USER,USING,VALUES,VAR,VARCHAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WEEK,WEEKS,WHEN,WHERE,WINDOW,WITH,WITHIN,X,YEAR,YEARS,ZONE") + assert(infoValue.getStringValue == "ADD,AFTER,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,AT,AUTHORIZATION,BETWEEN,BIGINT,BINARY,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHAR,CHARACTER,CHECK,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPUTE,CONCATENATE,CONSTRAINT,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,DATA,DATABASE,DATABASES,DATE,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAY,DAYOFYEAR,DAYS,DBPROPERTIES,DEC,DECIMAL,DECLARE,DEFAULT,DEFINED,DELETE,DELIMITED,DESC,DESCRIBE,DFS,DIRECTORIES,DIRECTORY,DISTINCT,DISTRIBUTE,DIV,DOUBLE,DROP,ELSE,END,ESCAPE,ESCAPED,EXCEPT,EXCHANGE,EXCLUDE,EXISTS,EXPLAIN,EXPORT,EXTENDED,EXTERNAL,EXTRACT,FALSE,FETCH,FIELDS,FILEFORMAT,FILTER,FIRST,FLOAT,FOLLOWING,FOR,FOREIGN,FORMAT,FORMATTED,FROM,FULL,FUNCTION,FUNCTIONS,GENERATED,GLOBAL,GRANT,GROUP,GROUPING,HAVING,HOUR,HOURS,IDENTIFIER,IF,IGNORE,ILIKE,IMPORT,IN,INCLUDE,INDEX,INDEXES,INNER,INPATH,INPUTFORMAT,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,IS,ITEMS,JOIN,KEYS,LAST,LATERAL,LAZY,LEADING,LEFT,LIKE,LIMIT,LINES,LIST,LOAD,LOCAL,LOCATION,LOCK,LOCKS,LOGICAL,LONG,MACRO,MAP,MATCHED,MERGE,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTE,MINUTES,MONTH,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NATURAL,NO,NULL,NULLS,NUMERIC,OF,OFFSET,ON,ONLY,OPTION,OPTIONS,OR,ORDER,OUT,OUTER,OUTPUTFORMAT,OVER,OVERLAPS,OVERLAY,OVERWRITE,PARTITION,PARTITIONED,PARTITIONS,PERCENT,PERCENTILE_CONT,PERCENTILE_DISC,PIVOT,PLACING,POSITION,PRECEDING,PRIMARY,PRINCIPALS,PROPERTIES,PURGE,QUARTER,QUERY,RANGE,REAL,RECORDREADER,RECORDWRITER,RECOVER,RECURSIVE,REDUCE,REFERENCES,REFRESH,RENAME,REPAIR,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,REVOKE,RIGHT,ROLE,ROLES,ROLLBACK,ROLLUP,ROW,ROWS,SCHEMA,SCHEMAS,SECOND,SECONDS,SELECT,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SESSION_USER,SET,SETS,SHORT,SHOW,SINGLE,SKEWED,SMALLINT,SOME,SORT,SORTED,SOURCE,START,STATISTICS,STORED,STRATIFY,STRING,STRUCT,SUBSTR,SUBSTRING,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLE,TABLES,TABLESAMPLE,TARGET,TBLPROPERTIES,TERMINATED,THEN,TIME,TIMEDIFF,TIMESTAMP,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TO,TOUCH,TRAILING,TRANSACTION,TRANSACTIONS,TRANSFORM,TRIM,TRUE,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNION,UNIQUE,UNKNOWN,UNLOCK,UNPIVOT,UNSET,UPDATE,USE,USER,USING,VALUES,VAR,VARCHAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WEEK,WEEKS,WHEN,WHERE,WINDOW,WITH,WITHIN,X,YEAR,YEARS,ZONE") // scalastyle:on line.size.limit } }