diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 7d6ee17ee8e13..eb4148beb2d39 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4447,6 +4447,18 @@ ], "sqlState" : "38000" }, + "RECURSION_LEVEL_LIMIT_EXCEEDED" : { + "message" : [ + "Recursion level limit reached but query has not exhausted, try increasing 'spark.sql.cteRecursionLevelLimit'" + ], + "sqlState" : "42836" + }, + "RECURSION_ROW_LIMIT_EXCEEDED" : { + "message" : [ + "Recursion row limit reached but query has not exhausted, try increasing 'spark.sql.cteRecursionRowLimit'" + ], + "sqlState" : "42836" + }, "RECURSIVE_CTE_IN_LEGACY_MODE" : { "message" : [ "Recursive definitions cannot be used in legacy CTE precedence mode (spark.sql.legacy.ctePrecedencePolicy=LEGACY)." @@ -5206,6 +5218,12 @@ ], "sqlState" : "42846" }, + "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE" : { + "message" : [ + "The UNION operator is not yet supported within recursive common table expressions (WITH clauses that refer to themselves, directly or indirectly). Please use UNION ALL instead." + ], + "sqlState" : "42836" + }, "UNKNOWN_PRIMITIVE_TYPE_IN_VARIANT" : { "message" : [ "Unknown primitive type with id was found in a variant value." 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 7d2c5b7d2e2d4..febb4c1282ac8 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 @@ -105,6 +105,9 @@ object ResolveWithCTE extends Rule[LogicalPlan] { // and we exclude those rows from the current iteration result. case alias @ SubqueryAlias(_, Distinct(Union(Seq(anchor, recursion), false, false))) => + cteDef.failAnalysis( + errorClass = "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + messageParameters = Map.empty) if (!anchor.resolved) { cteDef } else { @@ -126,6 +129,9 @@ object ResolveWithCTE extends Rule[LogicalPlan] { colNames, Distinct(Union(Seq(anchor, recursion), false, false)) )) => + cteDef.failAnalysis( + errorClass = "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + messageParameters = Map.empty) if (!anchor.resolved) { cteDef } else { 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 9d269f37e58b9..667e850af8b12 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 @@ -848,6 +848,13 @@ object LimitPushDown extends Rule[LogicalPlan] { case LocalLimit(exp, u: Union) => LocalLimit(exp, u.copy(children = u.children.map(maybePushLocalLimit(exp, _)))) + // If limit node is present, we should propagate it down to UnionLoop, so that it is later + // propagated to UnionLoopExec. + case LocalLimit(IntegerLiteral(limit), p @ Project(_, ul: UnionLoop)) => + p.copy(child = ul.copy(limit = Some(limit))) + case LocalLimit(IntegerLiteral(limit), ul: UnionLoop) => + ul.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. @@ -1032,6 +1039,10 @@ object ColumnPruning extends Rule[LogicalPlan] { 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) => val windowExprs = w.windowExpressions.filter(p.references.contains) 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 070090282fd02..88fb7ff865607 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 @@ -4537,6 +4537,22 @@ object SQLConf { .checkValues(LegacyBehaviorPolicy.values.map(_.toString)) .createWithDefault(LegacyBehaviorPolicy.CORRECTED.toString) + val CTE_RECURSION_LEVEL_LIMIT = buildConf("spark.sql.cteRecursionLevelLimit") + .doc("Maximum level of recursion that is allowed while executing a recursive CTE definition." + + "If a query does not get exhausted before reaching this limit it fails. Use -1 for " + + "unlimited.") + .version("4.1.0") + .intConf + .createWithDefault(100) + + val CTE_RECURSION_ROW_LIMIT = buildConf("spark.sql.cteRecursionRowLimit") + .doc("Maximum number of rows that can be returned when executing a recursive CTE definition." + + "If a query does not get exhausted before reaching this limit it fails. Use -1 for " + + "unlimited.") + .version("4.1.0") + .intConf + .createWithDefault(1000000) + 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/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 89f86c347568d..835ecddb45c8b 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 @@ -1031,6 +1031,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/UnionLoopExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala new file mode 100644 index 0000000000000..0694d02550a90 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala @@ -0,0 +1,230 @@ +/* + * 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.execution + +import scala.collection.mutable + +import org.apache.spark.SparkException +import org.apache.spark.rdd.{EmptyRDD, RDD} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical.{LocalLimit, LogicalPlan, Union, UnionLoopRef} +import org.apache.spark.sql.classic.Dataset +import org.apache.spark.sql.execution.LogicalRDD.rewriteStatsAndConstraints +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.internal.SQLConf + + +/** + * The physical node for recursion. Currently only UNION ALL case is supported. + * For the details about the execution, look at the comment above doExecute function. + * + * A simple recursive query: + * {{{ + * WITH RECURSIVE t(n) AS ( + * SELECT 1 + * UNION ALL + * SELECT n+1 FROM t WHERE n < 5) + * SELECT * FROM t; + * }}} + * Corresponding logical plan for the recursive query above: + * {{{ + * WithCTE + * :- CTERelationDef 0, false + * : +- SubqueryAlias t + * : +- Project [1#0 AS n#3] + * : +- UnionLoop 0 + * : :- Project [1 AS 1#0] + * : : +- OneRowRelation + * : +- Project [(n#1 + 1) AS (n + 1)#2] + * : +- Filter (n#1 < 5) + * : +- SubqueryAlias t + * : +- Project [1#0 AS n#1] + * : +- UnionLoopRef 0, [1#0], false + * +- Project [n#3] + * +- SubqueryAlias t + * +- CTERelationRef 0, true, [n#3], false, false + * }}} + * + * @param loopId This is id of the CTERelationDef containing the recursive query. Its value is + * first passed down to UnionLoop when creating it, and then to UnionLoopExec in + * SparkStrategies. + * @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. + * CTERelationRef, which is marked as recursive, gets substituted with + * [[UnionLoopRef]] in ResolveWithCTE. + * Both anchor and recursion are marked with @transient annotation, so that they + * are not serialized. + * @param output The output attributes of this loop. + * @param limit If defined, the total number of rows output by this operator will be bounded by + * limit. + * Its value is pushed down to UnionLoop in Optimizer in case LocalLimit node is + * present in the logical plan and then transferred to UnionLoopExec in + * SparkStrategies. + * Note here: limit can be applied in the main query calling the recursive CTE, and not + * inside the recursive term of recursive CTE. + */ +case class UnionLoopExec( + loopId: Long, + @transient anchor: LogicalPlan, + @transient recursion: LogicalPlan, + override val output: Seq[Attribute], + limit: Option[Int] = None) extends LeafExecNode { + + override def innerChildren: Seq[QueryPlan[_]] = Seq(anchor, recursion) + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numIterations" -> SQLMetrics.createMetric(sparkContext, "number of recursive iterations")) + + /** + * This function executes the plan (optionally with appended limit node) and caches the result, + * with the caching mode specified in config. + */ + private def executeAndCacheAndCount(plan: LogicalPlan, currentLimit: Int) = { + // In case limit is defined, we create a (local) limit node above the plan and execute + // the newly created plan. + val planWithLimit = if (limit.isDefined) { + LocalLimit(Literal(currentLimit), plan) + } else { + plan + } + val df = Dataset.ofRows(session, planWithLimit) + val materializedDF = df.repartition() + val count = materializedDF.queryExecution.toRdd.count() + (materializedDF, count) + } + + /** + * In the first iteration, anchor term is executed. + * Then, in each following iteration, the UnionLoopRef node is substituted with the plan from the + * previous iteration, and such plan is executed. + * After every iteration, the dataframe is materialized. + * The recursion stops when the generated dataframe is empty, or either the limit or + * the specified maximum depth from the config is reached. + */ + override protected def doExecute(): RDD[InternalRow] = { + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + val numOutputRows = longMetric("numOutputRows") + val numIterations = longMetric("numIterations") + val levelLimit = conf.getConf(SQLConf.CTE_RECURSION_LEVEL_LIMIT) + val rowLimit = conf.getConf(SQLConf.CTE_RECURSION_ROW_LIMIT) + + // currentLimit is initialized from the limit argument, and in each step it is decreased by + // the number of rows generated in that step. + // If limit is not passed down, currentLimit is set to be zero and won't be considered in the + // condition of while loop down (limit.isEmpty will be true). + var currentLimit = limit.getOrElse(-1) + + val unionChildren = mutable.ArrayBuffer.empty[LogicalRDD] + + var (prevDF, prevCount) = executeAndCacheAndCount(anchor, currentLimit) + + var currentLevel = 1 + + var currentNumRows = 0 + + var limitReached: Boolean = false + + val numPartitions = prevDF.queryExecution.toRdd.partitions.length + // Main loop for obtaining the result of the recursive query. + while (prevCount > 0 && !limitReached) { + + if (levelLimit != -1 && currentLevel > levelLimit) { + throw new SparkException( + errorClass = "RECURSION_LEVEL_LIMIT_EXCEEDED", + messageParameters = Map("levelLimit" -> levelLimit.toString), + cause = null) + } + + // Inherit stats and constraints from the dataset of the previous iteration. + val prevPlan = LogicalRDD.fromDataset(prevDF.queryExecution.toRdd, prevDF, prevDF.isStreaming) + .newInstance() + unionChildren += prevPlan + + currentNumRows += prevCount.toInt + + if (limit.isDefined) { + currentLimit -= prevCount.toInt + if (currentLimit <= 0) { + limitReached = true + } + } + + if (rowLimit != -1 && currentNumRows > rowLimit) { + throw new SparkException( + errorClass = "RECURSION_ROW_LIMIT_EXCEEDED", + messageParameters = Map("rowLimit" -> rowLimit.toString), + cause = null) + } + + // Update metrics + numOutputRows += prevCount + numIterations += 1 + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) + + if (!limitReached) { + // the current plan is created by substituting UnionLoopRef node with the project node of + // the previous plan. + // This way we support only UNION ALL case. Additional case should be added for UNION case. + // One way of supporting UNION case can be seen at SPARK-24497 PR from Peter Toth. + val newRecursion = recursion.transform { + case r: UnionLoopRef => + val logicalPlan = prevDF.logicalPlan + val optimizedPlan = prevDF.queryExecution.optimizedPlan + val (stats, constraints) = rewriteStatsAndConstraints(logicalPlan, optimizedPlan) + prevPlan.copy(output = r.output)(prevDF.sparkSession, stats, constraints) + } + + val (df, count) = executeAndCacheAndCount(newRecursion, currentLimit) + prevDF = df + prevCount = count + + currentLevel += 1 + } + } + + if (unionChildren.isEmpty) { + new EmptyRDD[InternalRow](sparkContext) + } else { + val df = { + if (unionChildren.length == 1) { + Dataset.ofRows(session, unionChildren.head) + } else { + Dataset.ofRows(session, Union(unionChildren.toSeq)) + } + } + val coalescedDF = df.coalesce(numPartitions) + coalescedDF.queryExecution.toRdd + } + } + + 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 + } +} 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..8096d06c47340 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out @@ -0,0 +1,1169 @@ +-- 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 +: +- 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 +: +- 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 +: +- Project [col1#x AS level#x] +: +- UnionLoopRef xxxx, [col1#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 +: +- 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 +: +- Project [a#x AS c#x] +: +- UnionLoopRef xxxx, [a#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 +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoopRef xxxx, [col1#x], false ++- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +CREATE TEMPORARY VIEW ZeroAndOne(current, next) AS VALUES + (0,0), + (0,1), + (1,0), + (1,1) +-- !query analysis +CreateViewCommand `ZeroAndOne`, [(current,None), (next,None)], VALUES + (0,0), + (0,1), + (1,0), + (1,1), false, false, LocalTempView, UNSUPPORTED, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT 1 + UNION ALL + SELECT next FROM t LEFT JOIN ZeroAndOne ON n = current +) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS n#x] +: +- UnionLoop xxxx +: :- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Project [next#x] +: +- Join LeftOuter, (n#x = current#x) +: :- SubqueryAlias t +: : +- Project [1#x AS n#x] +: : +- UnionLoopRef xxxx, [1#x], false +: +- SubqueryAlias zeroandone +: +- View (`ZeroAndOne`, [current#x, next#x]) +: +- Project [cast(col1#x as int) AS current#x, cast(col2#x as int) AS next#x] +: +- LocalRelation [col1#x, col2#x] ++- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false + + +-- !query +DROP VIEW ZeroAndOne +-- !query analysis +DropTempViewCommand ZeroAndOne + + +-- !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 +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoopRef xxxx, [col1#x], false ++- GlobalLimit 10 + +- LocalLimit 10 + +- Project [level#x] + +- SubqueryAlias r + +- CTERelationRef xxxx, true, [level#x], false, false + + +-- !query +WITH RECURSIVE r AS ( + SELECT 0 as level + UNION + SELECT (level + 1) % 10 FROM r +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 100, + "fragment" : "WITH RECURSIVE r AS (\n SELECT 0 as level\n UNION\n SELECT (level + 1) % 10 FROM r\n)\nSELECT * FROM r" + } ] +} + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION + SELECT (level + 1) % 10 FROM r +) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 98, + "fragment" : "WITH RECURSIVE r(level) AS (\n VALUES 0\n UNION\n SELECT (level + 1) % 10 FROM r\n)\nSELECT * FROM r" + } ] +} + + +-- !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 +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(level#x + 1) AS (level + 1)#x] +: +- SubqueryAlias r +: +- Project [col1#x AS level#x] +: +- UnionLoopRef xxxx, [col1#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" +} + + +-- !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 t(n) AS ( + SELECT 1 + UNION ALL + SELECT 2 +) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS n#x] +: +- Union false, false +: :- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Project [2 AS 2#x] +: +- OneRowRelation ++- Project [n#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [n#x], false, false, 2 + + +-- !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" +} + + +-- !query +WITH RECURSIVE t(col) ( + SELECT 1 + UNION ALL + SELECT (SELECT max(col) FROM t) +) +SELECT * FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836" +} + + +-- !query +WITH + t1 AS (SELECT 1 as n), + t2(n) AS ( + WITH RECURSIVE t3(n) AS ( + SELECT 1 + UNION ALL + SELECT n+1 FROM (SELECT MAX(n) FROM t3) + ) + SELECT * FROM t3 + ) +SELECT * FROM t2 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836" +} + + +-- !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 ALL ( + 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" : 159, + "stopIndex" : 160, + "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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 183, + "fragment" : "WITH RECURSIVE r(level, data) AS (\n VALUES (0, 0)\n UNION ALL\n SELECT r1.level + 1, r1.data\n FROM r AS r1\n JOIN r AS r2 ON r2.data = r1.data\n WHERE r1.level < 9\n)\nSELECT * FROM r" + } ] +} + + +-- !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 +: +- 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 +: : +- Project [col1#x AS level#x, col2#x AS data#x] +: : +- UnionLoopRef xxxx, [col1#x, col2#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" +} + + +-- !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 +: +- 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 +: +- Project [col1#x AS level#x, col2#x AS data#x] +: +- UnionLoopRef xxxx, [col1#x, col2#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" +} + + +-- !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" +} + + +-- !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 +: +- 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 +: : +- Project [col1#x AS level#x, col2#x AS data#x] +: : +- UnionLoopRef xxxx, [col1#x, col2#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" +} + + +-- !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 +: +- 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 +: : +- Project [col1#x AS level#x, col2#x AS data#x] +: : +- UnionLoopRef xxxx, [col1#x, col2#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" +} + + +-- !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" +} + + +-- !query +WITH RECURSIVE r(n) AS ( + SELECT 1 + UNION ALL + SELECT n+1 FROM r GROUP BY n) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836" +} + + +-- !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, UNSUPPORTED, true + +- WithCTE + :- CTERelationDef xxxx, false + : +- 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 + : +- Project [col1#x AS level#x] + : +- UnionLoopRef xxxx, [col1#x], false + +- 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 + : +- 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 + : +- Project [col1#x AS level#x] + : +- UnionLoopRef xxxx, [col1#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 + : +- 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 + : +- Project [col1#x AS level#x] + : +- UnionLoopRef xxxx, [col1#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 +: +- 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 +: +- 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 +: +- 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 +: +- 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 +: +- 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 +: +- 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 +: +- Project [col1#x AS level#x] +: +- UnionLoopRef xxxx, [col1#x], false +:- CTERelationDef xxxx, 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 +: +- 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 +: +- Project [col1#x AS level#x] +: +- UnionLoopRef xxxx, [col1#x], false +:- CTERelationDef xxxx, false +: +- 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 +: +- Project [level#x AS level#x] +: +- 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, UNSUPPORTED, 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 +: +- 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 +: +- SubqueryAlias fibonacci +: +- UnionLoop xxxx +: :- SubqueryAlias t +: : +- LocalRelation [a#x, 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 + + +-- !query +WITH RECURSIVE t1(a,b,c) AS ( + SELECT 1,1,1 + UNION ALL + SELECT a+1,a+1,a+1 FROM t1) +SELECT a FROM t1 LIMIT 5 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t1 +: +- Project [1#x AS a#x, 1#x AS b#x, 1#x AS c#x] +: +- UnionLoop xxxx +: :- Project [1 AS 1#x, 1 AS 1#x, 1 AS 1#x] +: : +- OneRowRelation +: +- Project [(a#x + 1) AS (a + 1)#x, (a#x + 1) AS (a + 1)#x, (a#x + 1) AS (a + 1)#x] +: +- SubqueryAlias t1 +: +- Project [1#x AS a#x, 1#x AS b#x, 1#x AS c#x] +: +- UnionLoopRef xxxx, [1#x, 1#x, 1#x], false ++- GlobalLimit 5 + +- LocalLimit 5 + +- Project [a#x] + +- SubqueryAlias t1 + +- CTERelationRef xxxx, true, [a#x, b#x, c#x], false, false 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 f9a282c2b927b..89ba8971a60d1 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 f9a282c2b927b..89ba8971a60d1 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/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out index 091b77eeaef4e..2fdced93217df 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 @@ -29,6 +29,288 @@ 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 +: +- 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 +: +- Project [col1#x AS n#x] +: +- UnionLoopRef xxxx, [col1#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 +: +- 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 +: +- Project [scalarsubquery()#x AS n#x] +: +- UnionLoopRef xxxx, [scalarsubquery()#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, UNSUPPORTED, true + +- WithCTE + :- CTERelationDef xxxx, false + : +- 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 + : +- Project [col1#x AS n#x] + : +- UnionLoopRef xxxx, [col1#x], false + +- 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 + : +- 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 + : +- Project [col1#x AS n#x] + : +- UnionLoopRef xxxx, [col1#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, UNSUPPORTED, true + +- WithCTE + :- CTERelationDef xxxx, false + : +- 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 + : +- Project [col1#x AS n#x] + : +- UnionLoopRef xxxx, [col1#x], false + +- 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 + : +- 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 + : +- Project [col1#x AS n#x] + : +- UnionLoopRef xxxx, [col1#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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "WITH RECURSIVE t(n) AS (\n SELECT 1\nUNION\n SELECT 10-n FROM t)\nSELECT * FROM t" + } ] +} + + +-- !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 +: +- SubqueryAlias t +: +- Project [col1#x AS n#x] +: +- UnionLoop xxxx +: :- LocalRelation [col1#x] +: +- Project [(n#x + 1) AS (n + 1)#x] +: +- SubqueryAlias t +: +- Project [col1#x AS n#x] +: +- UnionLoopRef xxxx, [col1#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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "WITH RECURSIVE t(n) AS (\n SELECT 1\nUNION\n SELECT n+1 FROM t)\nSELECT * FROM t LIMIT 10" + } ] +} + + +-- !query +WITH q AS (SELECT 'foo' AS x) +SELECT x FROM q +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias q +: +- Project [foo AS x#x] +: +- OneRowRelation ++- Project [x#x] + +- SubqueryAlias q + +- CTERelationRef xxxx, true, [x#x], false, false, 1 + + +-- !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 +: +- 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 +: +- Project [foo#x AS n#x] +: +- UnionLoopRef xxxx, [foo#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.SparkException +{ + "errorClass" : "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"BIGINT\"" + } +} + + -- !query CREATE TABLE department ( id INTEGER, -- department ID @@ -103,6 +385,380 @@ 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 +: +- 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, 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 +: +- 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 +: +- Project [1#x AS level#x, id#x AS id#x, parent_department#x AS parent_department#x, name#x AS name#x] +: +- UnionLoopRef xxxx, [1#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 +: +- 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 +: +- Project [1#x AS level#x, id#x AS id#x, parent_department#x AS parent_department#x, name#x AS name#x] +: +- UnionLoopRef xxxx, [1#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 +: +- 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 + : : +- 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 + : : +- Project [1#x AS n#x] + : : +- UnionLoopRef xxxx, [1#x], false + : +- Project [n#x] + : +- Filter (n#x < 50000) + : +- SubqueryAlias t + : +- CTERelationRef xxxx, true, [n#x], false, false + +- SubqueryAlias t + +- WithCTE + :- CTERelationDef xxxx, false + : +- 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 + : +- Project [1#x AS n#x] + : +- UnionLoopRef xxxx, [1#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 +: +- 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, UNSUPPORTED, true + +- WithCTE + :- CTERelationDef xxxx, false + : +- 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 +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 + : +- 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, COMPENSATION, true + +- WithCTE + :- CTERelationDef xxxx, false + : +- 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 + : +- Project [col1#x AS n#x] + : +- UnionLoopRef xxxx, [col1#x], false + +- 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 + : +- 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 + : +- Project [col1#x AS n#x] + : +- UnionLoopRef xxxx, [col1#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 +: +- 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 +: +- Project [col1#x AS i#x, col2#x AS j#x] +: +- UnionLoopRef xxxx, [col1#x, col2#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 +778,114 @@ 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 +: +- 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 +: +- Project [col1#x AS id#x, col2#x AS path#x] +: +- UnionLoopRef xxxx, [col1#x, col2#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 +: +- 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 +: +- Project [col1#x AS id#x, col2#x AS path#x] +: +- UnionLoopRef xxxx, [col1#x, col2#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 +: +- 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 +: +- Project [col1#x AS id#x, col2#x AS path#x] +: +- UnionLoopRef xxxx, [col1#x, col2#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,201 @@ 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 +: +- 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 +: +- 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] +: +- UnionLoopRef xxxx, [f#x, t#x, label#x, array(struct(f, t))#x, false#x], false ++- Project [f#x, t#x, label#x, path#x, cycle#x] + +- SubqueryAlias search_graph + +- CTERelationRef xxxx, true, [f#x, t#x, label#x, path#x, cycle#x], false, false + + +-- !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 +: +- 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 +: +- 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] +: +- UnionLoopRef xxxx, [f#x, t#x, label#x, array(struct(f, t))#x, false#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 +: +- SubqueryAlias y +: +- Project [col1#x AS id#x] +: +- LocalRelation [col1#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias x +: +- Project [id#x AS id#x] +: +- UnionLoop xxxx +: :- Project [id#x] +: : +- SubqueryAlias y +: : +- CTERelationRef xxxx, true, [id#x], false, false, 1 +: +- Project [(id#x + 1) AS (id + 1)#x] +: +- Filter (id#x < 5) +: +- SubqueryAlias x +: +- Project [id#x AS id#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 +: +- 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 +: +- Project [1#x AS id#x] +: +- UnionLoopRef xxxx, [1#x], false +:- CTERelationDef xxxx, 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 +: +- 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 +: +- Project [id#x AS id#x] +: +- 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 +: +- 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 +: +- Project [1#x AS id#x] +: +- UnionLoopRef xxxx, [1#x], false +:- CTERelationDef xxxx, 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 +: +- 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 +: +- Project [id#x AS id#x] +: +- 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 +1124,82 @@ 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" +} + + +-- !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" +} + + +-- !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" +} + + +-- !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" +} + + +-- !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" +} + + +-- !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,6 +1216,262 @@ 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" +} + + +-- !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" +} + + +-- !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" +} + + +-- !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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 134, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x\n WHERE n IN (SELECT * FROM x))\n SELECT * FROM x" + } ] +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT cast(1 as bigint) UNION ALL SELECT count(*) FROM x) + SELECT * FROM x +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836" +} + + +-- !query +WITH RECURSIVE x(n) AS (SELECT cast(1 as bigint) 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" +} + + +-- !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" +} + + +-- !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" +} + + +-- !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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 116, + "fragment" : "WITH RECURSIVE x(id) AS (values (1)\n UNION ALL\n SELECT (SELECT * FROM x) FROM x WHERE id < 5\n) SELECT * FROM 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.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`id`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 139, + "stopIndex" : 140, + "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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 175, + "fragment" : "WITH RECURSIVE foo(i) AS\n (values (1)\n UNION ALL\n (SELECT i+1 FROM foo WHERE i < 10\n UNION ALL\n SELECT i+1 FROM foo WHERE i < 5)\n) SELECT * FROM foo" + } ] +} + + +-- !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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 198, + "fragment" : "WITH RECURSIVE foo(i) AS\n (values (1)\n UNION ALL\n\t SELECT * FROM\n (SELECT i+1 FROM foo WHERE i < 10\n UNION ALL\n SELECT i+1 FROM foo WHERE i < 5) AS t\n) SELECT * FROM foo" + } ] +} + + +-- !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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 172, + "fragment" : "WITH RECURSIVE foo(i) AS\n (values (1)\n UNION ALL\n (SELECT i+1 FROM foo WHERE i < 10\n EXCEPT\n SELECT i+1 FROM foo WHERE i < 5)\n) SELECT * FROM foo" + } ] +} + + +-- !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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 175, + "fragment" : "WITH RECURSIVE foo(i) AS\n (values (1)\n UNION ALL\n (SELECT i+1 FROM foo WHERE i < 10\n INTERSECT\n SELECT i+1 FROM foo WHERE i < 5)\n) SELECT * FROM foo" + } ] +} + + +-- !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.SparkException +{ + "errorClass" : "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"DECIMAL(10,0)\"" + } +} + + -- !query with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q -- !query analysis @@ -197,6 +1488,47 @@ WithCTE +- CTERelationRef xxxx, true, [foo#x], false, false, 1 +-- !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 +: +- 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 +: +- Project [col1#x AS i#x] +: +- UnionLoopRef xxxx, [col1#x], false +:- CTERelationDef xxxx, false +: +- 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 +: +- Project [i#x AS j#x] +: +- UnionLoopRef xxxx, [i#x], false ++- Project [j#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [j#x], false, false + + -- !query WITH outermost(x) AS ( SELECT 1 @@ -257,6 +1589,51 @@ 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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 191, + "fragment" : "WITH RECURSIVE outermost(x) AS (\n SELECT 1\n UNION (WITH innermost as (SELECT 2)\n SELECT * FROM outermost\n UNION SELECT * FROM innermost)\n)\nSELECT * FROM outermost ORDER BY 1" + } ] +} + + +-- !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.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 185, + "fragment" : "WITH RECURSIVE outermost(x) AS (\n WITH innermost as (SELECT 2 FROM outermost) -- fail\n SELECT * FROM innermost\n UNION SELECT * from outermost\n)\nSELECT * FROM outermost ORDER BY 1" + } ] +} + + -- !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 @@ -477,6 +1854,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/inputs/cte-recursion.sql b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql new file mode 100644 index 0000000000000..b32c6f38ea1fd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql @@ -0,0 +1,465 @@ +--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.cteRecursionLevelLimit level +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r; + +-- unlimited recursion fails at spark.sql.cteRecursionRowLimit level +CREATE TEMPORARY VIEW ZeroAndOne(current, next) AS VALUES + (0,0), + (0,1), + (1,0), + (1,1); + +WITH RECURSIVE t(n) AS ( + SELECT 1 + UNION ALL + SELECT next FROM t LEFT JOIN ZeroAndOne ON n = current +) +SELECT * FROM t; + +DROP VIEW ZeroAndOne; + +-- terminate recursion with LIMIT +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r LIMIT 10; + +-- UNION - not yet supported +WITH RECURSIVE r AS ( + SELECT 0 as level + UNION + SELECT (level + 1) % 10 FROM r +) +SELECT * FROM r; + +-- UNION with subquery column alias - not yet supported +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; + +--recursive keyword but no self-reference +--other sql engines don't throw an error in this case, so Spark doesn't throw it as well +WITH RECURSIVE t(n) AS ( + SELECT 1 + UNION ALL + SELECT 2 +) +SELECT * FROM t; + +-- 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 t(col) ( + SELECT 1 + UNION ALL + SELECT (SELECT max(col) FROM t) +) +SELECT * FROM t; + +-- complicated subquery example: self-reference in subquery in an inner CTE +WITH + t1 AS (SELECT 1 as n), + t2(n) AS ( + WITH RECURSIVE t3(n) AS ( + SELECT 1 + UNION ALL + SELECT n+1 FROM (SELECT MAX(n) FROM t3) + ) + SELECT * FROM t3 + ) +SELECT * FROM t2; + + +-- recursive reference is not allowed in a nested CTE +-- TABLE_OR_VIEW_NOT_FOUND is thrown now, although it some check should be added to exactly inform +-- that this is not allowed +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 ALL ( + 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; + +-- group by inside recursion not allowed (considered as aggregate as well) +WITH RECURSIVE r(n) AS ( + SELECT 1 + UNION ALL + SELECT n+1 FROM r GROUP BY n) +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; + +-- Recursive CTE with useless columns +WITH RECURSIVE t1(a,b,c) AS ( + SELECT 1,1,1 + UNION ALL + SELECT a+1,a+1,a+1 FROM t1) +SELECT a FROM t1 LIMIT 5; \ No newline at end of file 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..bd349ec4a5329 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,118 +590,110 @@ 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; +-- we need to cast 1 as bigint, as otherwise it will throw an error that the data types +-- are different +WITH RECURSIVE x(n) AS (SELECT cast(1 as bigint) 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 cast(1 as bigint) 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 --WITH RECURSIVE foo(i) AS -- (SELECT i::numeric(3,0) FROM (VALUES(1),(2)) t(i) -- UNION ALL @@ -668,18 +730,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 +762,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 +796,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 +822,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 +854,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 +953,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 +1061,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 +1072,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 +1204,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 +1212,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 +1255,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/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out new file mode 100644 index 0000000000000..5f83ec7c1a85e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out @@ -0,0 +1,1052 @@ +-- 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 +{ + "errorClass" : "RECURSION_LEVEL_LIMIT_EXCEEDED", + "sqlState" : "42836", + "messageParameters" : { + "levelLimit" : "100" + } +} + + +-- !query +CREATE TEMPORARY VIEW ZeroAndOne(current, next) AS VALUES + (0,0), + (0,1), + (1,0), + (1,1) +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT 1 + UNION ALL + SELECT next FROM t LEFT JOIN ZeroAndOne ON n = current +) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "RECURSION_ROW_LIMIT_EXCEEDED", + "sqlState" : "42836", + "messageParameters" : { + "rowLimit" : "1000000" + } +} + + +-- !query +DROP VIEW ZeroAndOne +-- !query schema +struct<> +-- !query output + + + +-- !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 AS ( + SELECT 0 as level + UNION + SELECT (level + 1) % 10 FROM r +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 100, + "fragment" : "WITH RECURSIVE r AS (\n SELECT 0 as level\n UNION\n SELECT (level + 1) % 10 FROM r\n)\nSELECT * FROM r" + } ] +} + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES 0 + UNION + SELECT (level + 1) % 10 FROM r +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 98, + "fragment" : "WITH RECURSIVE r(level) AS (\n VALUES 0\n UNION\n SELECT (level + 1) % 10 FROM r\n)\nSELECT * FROM r" + } ] +} + + +-- !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 +{ + "errorClass" : "RECURSION_LEVEL_LIMIT_EXCEEDED", + "sqlState" : "42836", + "messageParameters" : { + "levelLimit" : "100" + } +} + + +-- !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" +} + + +-- !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 t(n) AS ( + SELECT 1 + UNION ALL + SELECT 2 +) +SELECT * FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !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" +} + + +-- !query +WITH RECURSIVE t(col) ( + SELECT 1 + UNION ALL + SELECT (SELECT max(col) FROM t) +) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836" +} + + +-- !query +WITH + t1 AS (SELECT 1 as n), + t2(n) AS ( + WITH RECURSIVE t3(n) AS ( + SELECT 1 + UNION ALL + SELECT n+1 FROM (SELECT MAX(n) FROM t3) + ) + SELECT * FROM t3 + ) +SELECT * FROM t2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836" +} + + +-- !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 ALL ( + 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" : 159, + "stopIndex" : 160, + "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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 183, + "fragment" : "WITH RECURSIVE r(level, data) AS (\n VALUES (0, 0)\n UNION ALL\n SELECT r1.level + 1, r1.data\n FROM r AS r1\n JOIN r AS r2 ON r2.data = r1.data\n WHERE r1.level < 9\n)\nSELECT * FROM r" + } ] +} + + +-- !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" +} + + +-- !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" +} + + +-- !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" +} + + +-- !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" +} + + +-- !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" +} + + +-- !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" +} + + +-- !query +WITH RECURSIVE r(n) AS ( + SELECT 1 + UNION ALL + SELECT n+1 FROM r GROUP BY n) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836" +} + + +-- !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 + + +-- !query +WITH RECURSIVE t1(a,b,c) AS ( + SELECT 1,1,1 + UNION ALL + SELECT a+1,a+1,a+1 FROM t1) +SELECT a FROM t1 LIMIT 5 +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 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 16077a78f3892..af7c4639b8410 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]: [col1#x] +Accumulated: false + +(4) Filter +Arguments: (col1#x < 9) + +(5) Project +Arguments: [(col1#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 9d25b829e03fc..5719c2620132f 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]: [col1#x] +Accumulated: false + +(4) Filter +Arguments: (col1#x < 9) + +(5) Project +Arguments: [(col1#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/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out index af1c33023a912..53d1be3b14473 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,204 @@ 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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "WITH RECURSIVE t(n) AS (\n SELECT 1\nUNION\n SELECT 10-n FROM t)\nSELECT * FROM t" + } ] +} + + +-- !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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "WITH RECURSIVE t(n) AS (\n SELECT 1\nUNION\n SELECT n+1 FROM t)\nSELECT * FROM t LIMIT 10" + } ] +} + + +-- !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.SparkException +{ + "errorClass" : "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"BIGINT\"" + } +} + + -- !query CREATE TABLE department ( id INTEGER, -- department ID @@ -28,13 +226,588 @@ CREATE TABLE department ( name string -- department name ) USING parquet -- !query schema -struct<> +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 -INSERT INTO department VALUES (0, NULL, 'ROOT') +CREATE TABLE y (a INTEGER) USING parquet -- !query schema struct<> -- !query output @@ -42,7 +815,7 @@ struct<> -- !query -INSERT INTO department VALUES (1, 0, 'A') +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) -- !query schema struct<> -- !query output @@ -50,7 +823,7 @@ struct<> -- !query -INSERT INTO department VALUES (2, 1, 'B') +DROP TABLE y -- !query schema struct<> -- !query output @@ -58,39 +831,95 @@ struct<> -- !query -INSERT INTO department VALUES (3, 2, 'C') +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" +} -- !query -INSERT INTO department VALUES (4, 2, 'D') +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" +} -- !query -INSERT INTO department VALUES (5, 0, 'E') +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" +} + +-- !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" +} -- !query -INSERT INTO department VALUES (6, 4, 'F') +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" +} + +-- !query +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 (7, 5, 'G') +CREATE TABLE y (a INTEGER) USING parquet -- !query schema struct<> -- !query output @@ -98,10 +927,7 @@ struct<> -- !query -CREATE TABLE tree( - id INTEGER, - parent_id INTEGER -) USING parquet +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) -- !query schema struct<> -- !query output @@ -109,75 +935,289 @@ struct<> -- !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 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" +} -- !query -create table graph( f int, t int, label string ) USING parquet +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" +} + +-- !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 schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE", + "sqlState" : "42836" +} -- !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(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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 134, + "fragment" : "WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x\n WHERE n IN (SELECT * FROM x))\n SELECT * FROM x" + } ] +} +-- !query +WITH RECURSIVE x(n) AS (SELECT cast(1 as bigint) 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" +} + -- !query -CREATE TABLE y (a INTEGER) USING parquet +WITH RECURSIVE x(n) AS (SELECT cast(1 as bigint) 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" +} +-- !query +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" +} + -- !query -INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +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" +} +-- !query +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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 116, + "fragment" : "WITH RECURSIVE x(id) AS (values (1)\n UNION ALL\n SELECT (SELECT * FROM x) FROM x WHERE id < 5\n) SELECT * FROM x" + } ] +} + -- !query -DROP TABLE y +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.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`id`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 139, + "stopIndex" : 140, + "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 schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 175, + "fragment" : "WITH RECURSIVE foo(i) AS\n (values (1)\n UNION ALL\n (SELECT i+1 FROM foo WHERE i < 10\n UNION ALL\n SELECT i+1 FROM foo WHERE i < 5)\n) SELECT * FROM foo" + } ] +} -- !query -CREATE TABLE y (a INTEGER) USING parquet +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.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 198, + "fragment" : "WITH RECURSIVE foo(i) AS\n (values (1)\n UNION ALL\n\t SELECT * FROM\n (SELECT i+1 FROM foo WHERE i < 10\n UNION ALL\n SELECT i+1 FROM foo WHERE i < 5) AS t\n) SELECT * FROM foo" + } ] +} + +-- !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 schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 172, + "fragment" : "WITH RECURSIVE foo(i) AS\n (values (1)\n UNION ALL\n (SELECT i+1 FROM foo WHERE i < 10\n EXCEPT\n SELECT i+1 FROM foo WHERE i < 5)\n) SELECT * FROM foo" + } ] +} -- !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 + INTERSECT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo -- !query schema struct<> -- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_RECURSIVE_REFERENCE.NUMBER", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 175, + "fragment" : "WITH RECURSIVE foo(i) AS\n (values (1)\n UNION ALL\n (SELECT i+1 FROM foo WHERE i < 10\n INTERSECT\n SELECT i+1 FROM foo WHERE i < 5)\n) SELECT * FROM foo" + } ] +} + +-- !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 schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"DECIMAL(10,0)\"" + } +} -- !query @@ -188,6 +1228,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 +1344,55 @@ 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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 191, + "fragment" : "WITH RECURSIVE outermost(x) AS (\n SELECT 1\n UNION (WITH innermost as (SELECT 2)\n SELECT * FROM outermost\n UNION SELECT * FROM innermost)\n)\nSELECT * FROM outermost ORDER BY 1" + } ] +} + + +-- !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.AnalysisException +{ + "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE", + "sqlState" : "42836", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 185, + "fragment" : "WITH RECURSIVE outermost(x) AS (\n WITH innermost as (SELECT 2 FROM outermost) -- fail\n SELECT * FROM innermost\n UNION SELECT * from outermost\n)\nSELECT * FROM outermost ORDER BY 1" + } ] +} + + -- !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 +1620,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/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index 9031a08686870..0c2380a2d90e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -769,6 +769,39 @@ class ParametersSuite extends QueryTest with SharedSparkSession { checkAnswer(spark.sql(query("?"), args = Array("t1")), Row(1)) } + test("SPARK-50892: parameterized identifier in outer query referencing a recursive CTE") { + def query(p: String): String = { + s""" + |WITH RECURSIVE t1(n) AS ( + | SELECT 1 + | UNION ALL + | SELECT n+1 FROM t1 WHERE n < 5) + |SELECT * FROM IDENTIFIER($p)""".stripMargin + } + + checkAnswer(spark.sql(query(":cte"), args = Map("cte" -> "t1")), + Seq(Row(1), Row(2), Row(3), Row(4), Row(5))) + checkAnswer(spark.sql(query("?"), args = Array("t1")), + Seq(Row(1), Row(2), Row(3), Row(4), Row(5))) + } + + test("SPARK-50892: parameterized identifier inside a recursive CTE") { + def query(p: String): String = { + s""" + |WITH RECURSIVE t1(n) AS ( + | SELECT 1 + | UNION ALL + | SELECT n+1 FROM IDENTIFIER($p) WHERE n < 5) + |SELECT * FROM t1""".stripMargin + } + + checkAnswer(spark.sql(query(":cte"), args = Map("cte" -> "t1")), + Seq(Row(1), Row(2), Row(3), Row(4), Row(5))) + checkAnswer(spark.sql(query("?"), args = Array("t1")), + Seq(Row(1), Row(2), Row(3), Row(4), Row(5))) + } + + test("SPARK-50403: parameterized execute immediate") { checkAnswer(spark.sql("execute immediate 'select ?' using ?", Array(1)), Row(1)) checkAnswer(spark.sql("execute immediate 'select ?, ?' using ?, 2", Array(1)), Row(1, 2)) 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 7cc556857774f..e807ae306ce76 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 @@ -59,6 +59,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+\\) ", "*") .replaceAll(