diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index e5ca7e9d10d59..3335d3fba843b 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -307,6 +307,7 @@ Below is a list of all the keywords in Spark SQL. |RECOVER|non-reserved|non-reserved|non-reserved| |REDUCE|non-reserved|non-reserved|non-reserved| |REFERENCES|reserved|non-reserved|reserved| +|RECURSIVE|reserved|non-reserved|reserved| |REFRESH|non-reserved|non-reserved|non-reserved| |RENAME|non-reserved|non-reserved|non-reserved| |REPAIR|non-reserved|non-reserved|non-reserved| diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 691fde8d48f94..0fdff6ee0fcbf 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -362,7 +362,7 @@ describeColName ; ctes - : WITH namedQuery (',' namedQuery)* + : WITH RECURSIVE? namedQuery (',' namedQuery)* ; namedQuery @@ -1386,6 +1386,7 @@ nonReserved | RECORDREADER | RECORDWRITER | RECOVER + | RECURSIVE | REDUCE | REFERENCES | REFRESH @@ -1643,6 +1644,7 @@ RANGE: 'RANGE'; RECORDREADER: 'RECORDREADER'; RECORDWRITER: 'RECORDWRITER'; RECOVER: 'RECOVER'; +RECURSIVE: 'RECURSIVE'; REDUCE: 'REDUCE'; REFERENCES: 'REFERENCES'; REFRESH: 'REFRESH'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9c97e1e9b441b..22e73f9b6489d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -218,6 +218,7 @@ class Analyzer( ResolveRelations :: ResolveTables :: ResolveReferences :: + ResolveRecursiveReferences :: ResolveCreateNamedStruct :: ResolveDeserializer :: ResolveNewInstance :: @@ -1657,6 +1658,23 @@ class Analyzer( } } + /** + * This rule resolve [[RecursiveReference]]s when the anchor term of the corresponding + * [[RecursiveRelation]] is resolved (ie. we know the output of the recursive relation). + */ + object ResolveRecursiveReferences extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case rr @ RecursiveRelation(cteName, anchorTerm, recursiveTerm) + if anchorTerm.resolved && !recursiveTerm.resolved => + + val newRecursiveTerm = recursiveTerm.transform { + case UnresolvedRecursiveReference(name, accumulated) if name == cteName => + RecursiveReference(name, anchorTerm.output.map(_.newInstance()), accumulated) + } + rr.copy(recursiveTerm = newRecursiveTerm) + } + } + /** * In many dialects of SQL it is valid to use ordinal positions in order/sort by and group by * clauses. This rule is to convert ordinal positions to the corresponding expressions in the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala index 9f0eff5017f38..03a7859a37287 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -21,6 +21,7 @@ import scala.collection.mutable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{Distinct, Except, LogicalPlan, RecursiveRelation, SubqueryAlias, Union, With} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, With} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf @@ -59,7 +60,7 @@ object CTESubstitution extends Rule[LogicalPlan] { startOfQuery: Boolean = true): Unit = { val resolver = SQLConf.get.resolver plan match { - case With(child, relations) => + case With(child, relations, _) => val newNames = mutable.ArrayBuffer.empty[String] newNames ++= outerCTERelationNames relations.foreach { @@ -86,7 +87,7 @@ object CTESubstitution extends Rule[LogicalPlan] { private def legacyTraverseAndSubstituteCTE(plan: LogicalPlan): LogicalPlan = { plan.resolveOperatorsUp { - case With(child, relations) => + case With(child, relations, _) => val resolvedCTERelations = resolveCTERelations(relations, isLegacy = true) substituteCTE(child, resolvedCTERelations) } @@ -135,20 +136,22 @@ object CTESubstitution extends Rule[LogicalPlan] { */ private def traverseAndSubstituteCTE(plan: LogicalPlan): LogicalPlan = { plan.resolveOperatorsUp { - case With(child: LogicalPlan, relations) => - val resolvedCTERelations = resolveCTERelations(relations, isLegacy = false) + case With(child: LogicalPlan, relations, allowRecursion) => + val resolvedCTERelations = resolveCTERelations(relations, isLegacy = false, allowRecursion) substituteCTE(child, resolvedCTERelations) case other => other.transformExpressions { - case e: SubqueryExpression => e.withNewPlan(traverseAndSubstituteCTE(e.plan)) + case e: SubqueryExpression => + e.withNewPlan(traverseAndSubstituteCTE(e.plan)) } } } private def resolveCTERelations( relations: Seq[(String, SubqueryAlias)], - isLegacy: Boolean): Seq[(String, LogicalPlan)] = { + isLegacy: Boolean, + allowRecursion: Boolean = false): Seq[(String, LogicalPlan)] = { val resolvedCTERelations = new mutable.ArrayBuffer[(String, LogicalPlan)](relations.size) for ((name, relation) <- relations) { val innerCTEResolved = if (isLegacy) { @@ -161,8 +164,13 @@ object CTESubstitution extends Rule[LogicalPlan] { // substitute CTE defined in `relation` first. traverseAndSubstituteCTE(relation) } + val recursionHandled = if (allowRecursion) { + handleRecursion(innerCTEResolved, name) + } else { + innerCTEResolved + } // CTE definition can reference a previous one - resolvedCTERelations += (name -> substituteCTE(innerCTEResolved, resolvedCTERelations)) + resolvedCTERelations += (name -> substituteCTE(recursionHandled, resolvedCTERelations)) } resolvedCTERelations } @@ -172,7 +180,7 @@ object CTESubstitution extends Rule[LogicalPlan] { cteRelations: Seq[(String, LogicalPlan)]): LogicalPlan = plan resolveOperatorsUp { case u @ UnresolvedRelation(Seq(table)) => - cteRelations.find(r => plan.conf.resolver(r._1, table)).map(_._2).getOrElse(u) + cteRelations.find(r => SQLConf.get.resolver(r._1, table)).map(_._2).getOrElse(u) case other => // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. @@ -180,4 +188,114 @@ object CTESubstitution extends Rule[LogicalPlan] { case e: SubqueryExpression => e.withNewPlan(substituteCTE(e.plan, cteRelations)) } } + + /** + * If recursion is allowed, recursion handling starts with inserting unresolved self-references + * ([[UnresolvedRecursiveReference]]) to places where a reference to the CTE definition itself is + * found. + * If there is a self-reference then we need to check if structure of the query satisfies the SQL + * recursion rules and insert a [[RecursiveRelation]] finally. + */ + private def handleRecursion(plan: LogicalPlan, cteName: String) = { + // check if there is any reference to the CTE and if there is then treat the CTE as recursive + val (recursiveReferencesPlan, recursiveReferenceCount) = + insertRecursiveReferences(plan, cteName) + if (recursiveReferenceCount > 0) { + // if there is a reference then the CTE needs to follow one of these structures + recursiveReferencesPlan match { + case SubqueryAlias(_, u: Union) => + insertRecursiveRelation(cteName, Seq.empty, false, u) + case SubqueryAlias(_, Distinct(u: Union)) => + insertRecursiveRelation(cteName, Seq.empty, true, u) + case SubqueryAlias(_, UnresolvedSubqueryColumnAliases(columnNames, u: Union)) => + insertRecursiveRelation(cteName, columnNames, false, u) + case SubqueryAlias(_, UnresolvedSubqueryColumnAliases(columnNames, Distinct(u: Union))) => + insertRecursiveRelation(cteName, columnNames, true, u) + case _ => + throw new AnalysisException(s"Recursive query $cteName should contain UNION or UNION " + + "ALL statements only. This error can also be caused by ORDER BY or LIMIT keywords " + + "used on result of UNION or UNION ALL.") + } + } else { + plan + } + } + + /** + * If we encounter a relation that matches the recursive CTE then the relation is replaced to an + * [[UnresolvedRecursiveReference]]. The replacement process also checks possible references in + * subqueries and reports them as errors. + */ + private def insertRecursiveReferences(plan: LogicalPlan, cteName: String): (LogicalPlan, Int) = { + val resolver = SQLConf.get.resolver + + var recursiveReferenceCount = 0 + val newPlan = plan resolveOperators { + case UnresolvedRelation(Seq(table)) if (resolver(cteName, table)) => + recursiveReferenceCount += 1 + UnresolvedRecursiveReference(cteName, false) + + case other => + other.subqueries.foreach(checkAndTraverse(_, { + case UnresolvedRelation(Seq(table)) if resolver(cteName, table) => + throw new AnalysisException(s"Recursive query $cteName should not contain recursive " + + "references in its subquery.") + case _ => true + })) + other + } + + (newPlan, recursiveReferenceCount) + } + + private def insertRecursiveRelation( + cteName: String, + columnNames: Seq[String], + distinct: Boolean, + union: Union) = { + if (union.children.size != 2) { + throw new AnalysisException(s"Recursive query ${cteName} should contain one anchor term " + + "and one recursive term connected with UNION or UNION ALL.") + } + + val anchorTerm :: recursiveTerm :: Nil = union.children + + // The anchor term shouldn't contain a recursive reference that matches the name of the CTE, + // except if it is nested under an other RecursiveRelation with the same name. + checkAndTraverse(anchorTerm, { + case UnresolvedRecursiveReference(name, _) if name == cteName => + throw new AnalysisException(s"Recursive query $cteName should not contain recursive " + + "references in its anchor (first) term.") + case RecursiveRelation(name, _, _) if name == cteName => false + case _ => true + }) + + // The anchor term has a special role, its output column are aliased if required. + val aliasedAnchorTerm = SubqueryAlias(cteName, + if (columnNames.nonEmpty) { + UnresolvedSubqueryColumnAliases(columnNames, anchorTerm) + } else { + anchorTerm + } + ) + + // If UNION combinator is used between the terms we extend the anchor with a DISTINCT and the + // recursive term with an EXCEPT clause and a reference to the so far accumulated result. + if (distinct) { + RecursiveRelation(cteName, Distinct(aliasedAnchorTerm), + Except(recursiveTerm, UnresolvedRecursiveReference(cteName, true), false)) + } else { + RecursiveRelation(cteName, aliasedAnchorTerm, recursiveTerm) + } + } + + /** + * Taverses the plan including subqueries and run the check while it returns true. + */ + private def checkAndTraverse(plan: LogicalPlan, check: LogicalPlan => Boolean): Unit = { + if (check(plan)) { + plan.children.foreach(checkAndTraverse(_, check)) + plan.subqueries.foreach(checkAndTraverse(_, check)) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 9e325d0c2e4e1..0891bc82f3e3b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -654,6 +654,7 @@ trait CheckAnalysis extends PredicateHelper { case _ => // Analysis successful! } } + checkRecursion(plan) checkCollectedMetrics(plan) extendedCheckRules.foreach(_(plan)) plan.foreachUp { @@ -665,6 +666,62 @@ trait CheckAnalysis extends PredicateHelper { plan.setAnalyzed() } + /** + * Recursion according to SQL standard comes with several limitations due to the fact that only + * those operations are allowed where the new set of rows can be computed from the result of the + * previous iteration. This implies that a recursive reference can't be used in some kinds of + * joins and aggregations. + * A further constraint is that a recursive term can contain one recursive reference only (except + * for using it on different sides of a UNION). + * + * This rule checks that these restrictions are not violated and returns the original plan. + */ + private def checkRecursion( + plan: LogicalPlan, + allowedRecursiveReferencesAndCounts: mutable.Map[String, Int] = mutable.Map.empty): Unit = { + plan match { + case RecursiveRelation(name, anchorTerm, recursiveTerm) => + if (allowedRecursiveReferencesAndCounts.contains(name)) { + throw new AnalysisException(s"Recursive CTE definition $name is already in use.") + } + checkRecursion(anchorTerm, allowedRecursiveReferencesAndCounts) + checkRecursion(recursiveTerm, allowedRecursiveReferencesAndCounts += name -> 0) + allowedRecursiveReferencesAndCounts -= name + case RecursiveReference(name, _, false, _, _, _) => + if (!allowedRecursiveReferencesAndCounts.contains(name)) { + throw new AnalysisException(s"Recursive reference $name cannot be used here. This can " + + "be caused by using it on inner side of an outer join, using it with aggregate in a " + + "subquery or using it multiple times in a recursive term (except for using it on " + + "different sides of an UNION ALL).") + } + if (allowedRecursiveReferencesAndCounts(name) > 0) { + throw new AnalysisException(s"Recursive reference $name cannot be used multiple times " + + "in a recursive term.") + } + + allowedRecursiveReferencesAndCounts += + name -> (allowedRecursiveReferencesAndCounts(name) + 1) + case Join(left, right, Inner, _, _) => + checkRecursion(left, allowedRecursiveReferencesAndCounts) + checkRecursion(right, allowedRecursiveReferencesAndCounts) + case Join(left, right, LeftOuter, _, _) => + checkRecursion(left, allowedRecursiveReferencesAndCounts) + checkRecursion(right, mutable.Map.empty) + case Join(left, right, RightOuter, _, _) => + checkRecursion(left, mutable.Map.empty) + checkRecursion(right, allowedRecursiveReferencesAndCounts) + case Join(left, right, _, _, _) => + checkRecursion(left, mutable.Map.empty) + checkRecursion(right, mutable.Map.empty) + case Aggregate(_, _, child) => checkRecursion(child, mutable.Map.empty) + case Union(children) => + children.foreach(checkRecursion(_, + mutable.Map(allowedRecursiveReferencesAndCounts.keys.map(name => name -> 0).toSeq: _*))) + case o => + o.children.foreach(checkRecursion(_, allowedRecursiveReferencesAndCounts)) + } + } + /** * Validates subquery expressions in the plan. Upon failure, returns an user facing error. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index b28be042c43f5..93e2ffee78447 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -550,3 +550,9 @@ case class UnresolvedHaving( override lazy val resolved: Boolean = false override def output: Seq[Attribute] = child.output } + +case class UnresolvedRecursiveReference(cteName: String, accumulated: Boolean) extends LeafNode { + override def output: Seq[Attribute] = Nil + + override lazy val resolved = false +} 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 e800ee3b93f51..262828086e353 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 @@ -651,6 +651,9 @@ object ColumnPruning extends Rule[LogicalPlan] { case NestedColumnAliasing(p) => p + // Don't prune columns of RecursiveTable + case p @ Project(_, _: RecursiveRelation) => p + // for all other logical plans that inherits the output from it's children // Project over project is handled by the first case, skip it here. case p @ Project(_, child) if !child.isInstanceOf[Project] => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 03571a740df3e..9d08bf894f9e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -141,7 +141,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging s"CTE definition can't have duplicate names: ${duplicates.mkString("'", "', '", "'")}.", ctx) } - With(plan, ctes) + With(plan, ctes, ctx.RECURSIVE() != null) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 79a8380826ab3..bb369aab636b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -17,10 +17,9 @@ package org.apache.spark.sql.catalyst.plans.logical -import scala.collection.mutable - -import org.apache.spark.sql.catalyst.AliasIdentifier -import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.{AliasIdentifier, InternalRow} +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction} @@ -28,6 +27,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.random.RandomSampler @@ -58,6 +58,62 @@ object Subquery { Subquery(s.plan, SubqueryExpression.hasCorrelatedSubquery(s)) } +/** + * This node defines a relation that contains a [[RecursiveReference]]s as child nodes referring to + * the relation itself. It contains one anchor and one recursive term as children and can be used to + * define a recursive query. The result of the anchor and the repeatedly executed recursive terms + * are combined using UNION or UNION ALL to form the final result. + * + * @param cteName name of the recursive relation + * @param anchorTerm this child is used for initializing the query + * @param recursiveTerm this child is used for extending the results with new rows based on the + * results of the previous iteration (or based on the results of the anchor in + * the first iteration) + */ +case class RecursiveRelation( + cteName: String, + anchorTerm: LogicalPlan, + recursiveTerm: LogicalPlan) extends UnionBase { + override def children: Seq[LogicalPlan] = anchorTerm :: recursiveTerm :: Nil +} + +/** + * This node is a reference to a recursive relation in CTE definitions. + * + * It is important that we can't calculate the statistics of a [[RecursiveRelation]] before the + * execution. And can't even estimate it as the recursive term can vastly increase the number of + * rows returned. To stay on the safe side [[RecursiveReference]] returns defaultSizeInBytes by + * default which causes it's [[RecursiveRelation]] statistics to be defaultSizeInBytes as well. + * + * Please note that during recursive execution the statistics and data are refreshed based on the + * results of the previous iteration to recreate the best physical plan in each iteration. + * + * @param cteName the name of the table it references to + * @param output the attributes of the recursive relation + * @param accumulated defines if the reference carries accumulated result + * @param level the recursion level + * @param statistics statistics of the data that this reference caries + * @param data data that this reference caries + */ +case class RecursiveReference( + cteName: String, + output: Seq[Attribute], + accumulated: Boolean, + level: Int = 0, + statistics: Statistics = Statistics(SQLConf.get.defaultSizeInBytes), + data: RDD[InternalRow] = null) extends LeafNode { + override def computeStats(): Statistics = statistics + + override def stringArgs: Iterator[Any] = Iterator(cteName, output, accumulated, level) + + def withNewIteration( + level: Int, + statistics: Statistics, + data: RDD[InternalRow]): RecursiveReference = { + copy(level = level, statistics = statistics, data = data) + } +} + case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends OrderPreservingUnaryNode { override def output: Seq[Attribute] = projectList.map(_.toAttribute) @@ -217,10 +273,40 @@ object Union { } } +abstract class UnionBase extends LogicalPlan { + // updating nullability to make all the children consistent + override def output: Seq[Attribute] = { + children.map(_.output).transpose.map { attrs => + val firstAttr = attrs.head + val nullable = attrs.exists(_.nullable) + val newDt = attrs.map(_.dataType).reduce(StructType.merge) + if (firstAttr.dataType == newDt) { + firstAttr.withNullability(nullable) + } else { + AttributeReference(firstAttr.name, newDt, nullable, firstAttr.metadata)( + firstAttr.exprId, firstAttr.qualifier) + } + } + } + + override lazy val resolved: Boolean = { + // allChildrenCompatible needs to be evaluated after childrenResolved + def allChildrenCompatible: Boolean = + children.tail.forall( child => + // compare the attribute number with the first child + child.output.length == children.head.output.length && + // compare the data types with the first child + child.output.zip(children.head.output).forall { + case (l, r) => l.dataType.sameType(r.dataType) + }) + children.length > 1 && childrenResolved && allChildrenCompatible + } +} + /** * Logical plan for unioning two plans, without a distinct. This is UNION ALL in SQL. */ -case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { +case class Union(children: Seq[LogicalPlan]) extends UnionBase { override def maxRows: Option[Long] = { if (children.exists(_.maxRows.isEmpty)) { None @@ -245,34 +331,6 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { AttributeSet.fromAttributeSets(children.map(_.outputSet)).size } - // updating nullability to make all the children consistent - override def output: Seq[Attribute] = { - children.map(_.output).transpose.map { attrs => - val firstAttr = attrs.head - val nullable = attrs.exists(_.nullable) - val newDt = attrs.map(_.dataType).reduce(StructType.merge) - if (firstAttr.dataType == newDt) { - firstAttr.withNullability(nullable) - } else { - AttributeReference(firstAttr.name, newDt, nullable, firstAttr.metadata)( - firstAttr.exprId, firstAttr.qualifier) - } - } - } - - override lazy val resolved: Boolean = { - // allChildrenCompatible needs to be evaluated after childrenResolved - def allChildrenCompatible: Boolean = - children.tail.forall( child => - // compare the attribute number with the first child - child.output.length == children.head.output.length && - // compare the data types with the first child - child.output.zip(children.head.output).forall { - case (l, r) => l.dataType.sameType(r.dataType) - }) - children.length > 1 && childrenResolved && allChildrenCompatible - } - /** * Maps the constraints containing a given (original) sequence of attributes to those with a * given (reference) sequence of attributes. Given the nature of union, we expect that the @@ -453,12 +511,16 @@ case class View( * @param cteRelations A sequence of pair (alias, the CTE definition) that this CTE defined * Each CTE can see the base tables and the previously defined CTEs only. */ -case class With(child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { +case class With( + child: LogicalPlan, + cteRelations: Seq[(String, SubqueryAlias)], + allowRecursion: Boolean = false) extends UnaryNode { override def output: Seq[Attribute] = child.output override def simpleString(maxFields: Int): String = { val cteAliases = truncatedString(cteRelations.map(_._1), "[", ", ", "]", maxFields) - s"CTE $cteAliases" + val recursive = if (allowRecursion) " recursive" else "" + s"CTE$recursive $cteAliases" } override def innerChildren: Seq[LogicalPlan] = cteRelations.map(_._2) 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 7f63d79a21ed6..904f9de034195 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 @@ -2452,6 +2452,14 @@ object SQLConf { .checkValues(LegacyBehaviorPolicy.values.map(_.toString)) .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString) + val CTE_RECURSION_LEVEL_LIMIT = buildConf("spark.sql.cte.recursion.level.limit") + .internal() + .doc("Maximum level of recursion that is allowed wile executing a recursive CTE definition." + + "If a query does not get exhausted before reaching this limit it fails. Use -1 for " + + "unlimited.") + .intConf + .createWithDefault(100) + val LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC = buildConf("spark.sql.legacy.followThreeValuedLogicInArrayExists") .internal() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 88afcb10d9c20..bce8c013882cf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -42,7 +42,10 @@ class PlanParserSuite extends AnalysisTest { private def intercept(sqlCommand: String, messages: String*): Unit = interceptParseException(parsePlan)(sqlCommand, messages: _*) - private def cte(plan: LogicalPlan, namedPlans: (String, (LogicalPlan, Seq[String]))*): With = { + private def cte( + plan: LogicalPlan, + allowRecursion: Boolean, + namedPlans: (String, (LogicalPlan, Seq[String]))*): With = { val ctes = namedPlans.map { case (name, (cte, columnAliases)) => val subquery = if (columnAliases.isEmpty) { @@ -52,7 +55,7 @@ class PlanParserSuite extends AnalysisTest { } name -> SubqueryAlias(name, subquery) } - With(plan, ctes) + With(plan, ctes, allowRecursion) } test("single comment case one") { @@ -192,13 +195,13 @@ class PlanParserSuite extends AnalysisTest { test("common table expressions") { assertEqual( "with cte1 as (select * from a) select * from cte1", - cte(table("cte1").select(star()), "cte1" -> ((table("a").select(star()), Seq.empty)))) + cte(table("cte1").select(star()), false, "cte1" -> ((table("a").select(star()), Seq.empty)))) assertEqual( "with cte1 (select 1) select * from cte1", - cte(table("cte1").select(star()), "cte1" -> ((OneRowRelation().select(1), Seq.empty)))) + cte(table("cte1").select(star()), false, "cte1" -> ((OneRowRelation().select(1), Seq.empty)))) assertEqual( "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2", - cte(table("cte2").select(star()), + cte(table("cte2").select(star()), false, "cte1" -> ((OneRowRelation().select(1), Seq.empty)), "cte2" -> ((table("cte1").select(star()), Seq.empty)))) intercept( @@ -1011,7 +1014,7 @@ class PlanParserSuite extends AnalysisTest { |WITH cte1 AS (SELECT * FROM testcat.db.tab) |SELECT * FROM cte1 """.stripMargin, - cte(table("cte1").select(star()), + cte(table("cte1").select(star()), false, "cte1" -> ((table("testcat", "db", "tab").select(star()), Seq.empty)))) assertEqual( @@ -1022,7 +1025,20 @@ class PlanParserSuite extends AnalysisTest { test("CTE with column alias") { assertEqual( "WITH t(x) AS (SELECT c FROM a) SELECT * FROM t", - cte(table("t").select(star()), "t" -> ((table("a").select('c), Seq("x"))))) + cte(table("t").select(star()), false, "t" -> ((table("a").select('c), Seq("x"))))) + } + + test("Recursive CTE") { + assertEqual( + """WITH RECURSIVE t(x) AS ( + | SELECT 0 AS level, c FROM a + | UNION ALL + | SELECT level + 1, c FROM t WHERE level < 10 + |) + |SELECT * FROM t""".stripMargin, + cte(table("t").select(star()), true, + "t" -> ((table("a").select(Literal(0).as("level"), 'c) + .union(table("t").where('level < 10).select('level + 1, 'c)), Seq("x"))))) } test("statement containing terminal semicolons") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 04c427da3ec87..8266c33652b91 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -396,7 +396,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { val reservedKeywordsInAnsiMode = allCandidateKeywords -- nonReservedKeywordsInAnsiMode test("check # of reserved keywords") { - val numReservedKeywords = 74 + val numReservedKeywords = 75 assert(reservedKeywordsInAnsiMode.size == numReservedKeywords, s"The expected number of reserved keywords is $numReservedKeywords, but " + s"${reservedKeywordsInAnsiMode.size} found.") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 53bb0e3a527cc..f6d90f4095a00 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -120,6 +120,11 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => splitConjunctivePredicates(condition.get).map(rewriteBinaryComparison) .sortBy(_.hashCode()).reduce(And) Join(left, right, newJoinType, Some(newCondition), hint) + case w @ With(_, cteRelations, _) => + w.copy(cteRelations = cteRelations.map { + case (cteName, ctePlan) => + cteName -> normalizePlan(normalizeExprIds(ctePlan)).asInstanceOf[SubqueryAlias] + }) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 302aae08d588b..088a1c8d83dbe 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 @@ -644,6 +644,11 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.ProjectExec(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.FilterExec(condition, planLater(child)) :: Nil + case rr @ logical.RecursiveRelation(name, anchorTerm, recursiveTerm) => + execution.RecursiveRelationExec(name, planLater(anchorTerm), recursiveTerm, rr.output) :: + Nil + case logical.RecursiveReference(name, output, _, level, _, rdd) => + RDDScanExec(output, rdd, s"RecursiveReference $name, $level") :: Nil case f: logical.TypedFilter => execution.FilterExec(f.typedCondition(f.deserializer), planLater(f.child)) :: Nil case e @ logical.Expand(_, _, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 4b376b94566b8..fc2111b62efaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -21,17 +21,21 @@ import java.util.concurrent.{Future => JFuture} import java.util.concurrent.TimeUnit._ import scala.collection.mutable -import scala.concurrent.{ExecutionContext} +import scala.concurrent.ExecutionContext import scala.concurrent.duration.Duration -import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext} +import org.apache.spark._ import org.apache.spark.rdd.{EmptyRDD, PartitionwiseSampledRDD, RDD} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, RecursiveReference, RecursiveRelation, Statistics} +import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.util.{ThreadUtils, Utils} @@ -253,6 +257,107 @@ case class FilterExec(condition: Expression, child: SparkPlan) } } +/** + * Physical plan node for a recursive relation that encapsulates the physical plan of the anchor + * term and the logical plan of the recursive term. + * + * Anchor is used to initialize the query in the first run. + * Recursive term is used to extend the result with new rows, They are logical plans and contain + * references to the result of the previous iteration or to the so far accumulated result. These + * references are updated with new statistics and data and then compiled to physical plan before + * execution. + * + * The execution terminates once the anchor term or an iteration of the recursive term return no + * rows. + * + * @param cteName the name of the recursive relation + * @param anchorTerm this child is used for initializing the query + * @param output the attributes of the recursive relation + */ +case class RecursiveRelationExec( + cteName: String, + anchorTerm: SparkPlan, + @transient logicalRecursiveTerm: LogicalPlan, + override val output: Seq[Attribute]) extends SparkPlan { + override def children: Seq[SparkPlan] = Seq(anchorTerm) + + override def innerChildren: Seq[QueryPlan[_]] = logicalRecursiveTerm +: super.innerChildren + + override def stringArgs: Iterator[Any] = Iterator(cteName, output) + + private var physicalRecursiveTerms = new mutable.ArrayBuffer[SparkPlan] + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + + private def calculateStatistics(count: Long) = { + Statistics(EstimationUtils.getSizePerRow(output) * count, Some(count)) + } + + private def unionRDDs(rdds: Seq[RDD[InternalRow]]): RDD[InternalRow] = { + if (rdds.size == 1) { + rdds.head + } else { + sparkContext.union(rdds) + } + } + + override protected def doExecute(): RDD[InternalRow] = { + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + val levelLimit = conf.getConf(SQLConf.CTE_RECURSION_LEVEL_LIMIT) + + // TODO: cache before count, as the RDD can be reused in the next iteration + var prevIterationRDD = anchorTerm.execute().map(_.copy()) + var prevIterationCount = prevIterationRDD.count() + + val accumulatedRDDs = mutable.ArrayBuffer.empty[RDD[InternalRow]] + var accumulatedCount = 0L + + val numOutputRows = longMetric("numOutputRows") + + var level = 0 + while (prevIterationCount > 0) { + if (levelLimit != -1 && level > levelLimit) { + throw new SparkException(s"Recursion level limit ${levelLimit} reached but query has not " + + s"exhausted, try increasing ${SQLConf.CTE_RECURSION_LEVEL_LIMIT.key}") + } + + accumulatedRDDs += prevIterationRDD + accumulatedCount += prevIterationCount + + numOutputRows += prevIterationCount + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) + + val newLogicalRecursiveTerm = logicalRecursiveTerm.transform { + case rr @ RecursiveReference(name, _, accumulated, _, _, _) if name == cteName => + val (newStatistics, newRDD) = if (accumulated) { + (calculateStatistics(accumulatedCount), unionRDDs(accumulatedRDDs)) + } else { + (calculateStatistics(prevIterationCount), prevIterationRDD) + } + rr.withNewIteration(level, newStatistics, newRDD) + } + + val physicalRecursiveTerm = + QueryExecution.prepareExecutedPlan(sqlContext.sparkSession, newLogicalRecursiveTerm) + + physicalRecursiveTerms += physicalRecursiveTerm + + // TODO: cache before count, as the RDD can be reused in the next iteration + prevIterationRDD = physicalRecursiveTerm.execute().map(_.copy()) + prevIterationCount = prevIterationRDD.count() + + level = level + 1 + } + + if (accumulatedRDDs.isEmpty) { + prevIterationRDD + } else { + unionRDDs(accumulatedRDDs) + } + } +} + /** * Physical plan for sampling the dataset. * 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..522162f607f94 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql @@ -0,0 +1,409 @@ +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1 + +--CONFIG_DIM2 spark.sql.adaptive.enabled=false +--CONFIG_DIM2 spark.sql.adaptive.enabled=true + +CREATE TEMPORARY VIEW t AS SELECT * FROM VALUES 0, 1, 2 AS t(id); + +-- fails due to recursion isn't allowed with RECURSIVE keyword +WITH r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 +) +SELECT * FROM r; + +-- very basic recursion +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 +) +SELECT * FROM r; + +-- unlimited recursion fails at spark.sql.cte.recursion.level.limits level +SET spark.sql.cte.recursion.level.limit=100; +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r; + +-- terminate recursion with LIMIT +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r LIMIT 10; + +-- terminate projected recursion with LIMIT +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r +) +SELECT level, level FROM r LIMIT 10; + +-- fails because using LIMIT to terminate recursion only works where Limit can be pushed through +-- recursion +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r +) +SELECT level, level FROM r ORDER BY level LIMIT 10; + +-- using string column in recursion +WITH RECURSIVE r(c) AS ( + SELECT 'a' + UNION ALL + SELECT c || ' b' FROM r WHERE LENGTH(c) < 10 +) +SELECT * FROM r; + +-- recursion works regardless the order of anchor and recursive terms +WITH RECURSIVE r(level) AS ( + SELECT level + 1 FROM r WHERE level < 10 + UNION ALL + VALUES (0) +) +SELECT * FROM r; + +-- multiple anchor terms are not supported +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + VALUES (0, 'B') + UNION ALL + SELECT level + 1, data || 'C' FROM r WHERE level < 3 +) +SELECT * FROM r; + +-- multiple recursive terms are not supported +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT level + 1, data || 'B' FROM r WHERE level < 2 + UNION ALL + SELECT level + 1, data || 'C' FROM r WHERE level < 3 +) +SELECT * FROM r; + +-- multiple anchor and recursive terms are not supported +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + VALUES (0, 'B') + UNION ALL + SELECT level + 1, data || 'C' FROM r WHERE level < 2 + UNION ALL + SELECT level + 1, data || 'D' FROM r WHERE level < 3 +) +SELECT * FROM r; + +-- recursion without an anchor term fails +WITH RECURSIVE r(level) AS ( + SELECT level + 1 FROM r WHERE level < 3 +) +SELECT * FROM r; + +-- UNION combinator supported to eliminate duplicates and stop recursion +WITH RECURSIVE r(level) AS ( + VALUES (0), (0) + UNION + SELECT (level + 1) % 10 FROM r +) +SELECT * FROM r; + +-- fails because a recursive query should contain UNION ALL or UNION combinator +WITH RECURSIVE r(level) AS ( + VALUES (0) + INTERSECT + SELECT level + 1 FROM r WHERE level < 10 +) +SELECT * FROM r; + +-- recursive reference is not allowed in a subquery +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE (SELECT SUM(level) FROM r) < 10 +) +SELECT * FROM r; + +-- recursive reference can't be used multiple times in a recursive term +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT r1.level + 1, r1.data + FROM r AS r1 + JOIN r AS r2 ON r2.data = r1.data + WHERE r1.level < 10 +) +SELECT * FROM r; + +-- recursive reference is not allowed on right side of a left outer join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT level + 1, r.data + FROM ( + SELECT 'B' AS data + ) AS o + LEFT JOIN r ON r.data = o.data +) +SELECT * FROM r; + +-- recursive reference is not allowed on left side of a right outer join +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT level + 1, r.data + FROM r + RIGHT JOIN ( + SELECT 'B' AS data + ) AS o ON o.data = r.data +) +SELECT * FROM r; + +-- aggregate is supported in the anchor term +WITH RECURSIVE r(level, data) AS ( + SELECT MAX(level) AS level, SUM(data) AS data FROM VALUES (0, 1), (0, 2) + UNION ALL + SELECT level + 1, data FROM r WHERE level < 10 +) +SELECT * FROM r ORDER BY level; + +-- recursive reference is not allowed in an aggregate in a recursive term +WITH RECURSIVE r(id, data) AS ( + VALUES (0, 1L) + UNION ALL + SELECT 1, SUM(data) FROM r WHERE data < 10 GROUP BY id +) +SELECT * FROM r; + +-- recursive reference is not allowed in an aggregate (made from project) in a recursive term +WITH RECURSIVE r(level) AS ( + VALUES (1L) + UNION ALL + SELECT SUM(level) FROM r WHERE level < 10 +) +SELECT * FROM r; + +-- aggregate is supported on a recursive relation +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT level + 1, data FROM r WHERE level < 10 +) +SELECT COUNT(*) FROM r; + +-- recursive reference within distinct is supported +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT DISTINCT level + 1, data FROM r WHERE level < 10 +) +SELECT * FROM r; + +-- multiple with works +WITH RECURSIVE y AS ( + VALUES (1) AS t(id) +), +x AS ( + SELECT * FROM y + UNION ALL + SELECT id + 1 FROM x WHERE id < 5 +) +SELECT * FROM x; + +-- multiple with works 2 +WITH RECURSIVE x AS ( + VALUES (1) AS t(id) + UNION ALL + SELECT id + 1 FROM x WHERE id < 5 +), +y AS ( + VALUES (1) AS t(id) + UNION ALL + SELECT id + 1 FROM y WHERE id < 10 +) +SELECT * FROM y LEFT JOIN x ON x.id = y.id; + +-- multiple with works 3 +WITH RECURSIVE x AS ( + VALUES (1) AS t(id) + UNION ALL + SELECT id + 1 FROM x WHERE id < 5 +), +y AS ( + VALUES (1) AS t(id) + UNION ALL + SELECT id + 1 FROM x WHERE id < 10 +) +SELECT * FROM y LEFT JOIN x ON x.id = y.id; + +-- multiple with works 4 +WITH RECURSIVE x AS ( + SELECT 1 AS id + UNION ALL + SELECT id + 1 FROM x WHERE id < 3 +), +y AS ( + SELECT * FROM x + UNION ALL + SELECT * FROM x +), +z AS ( + SELECT * FROM x + UNION ALL + SELECT id + 1 FROM z WHERE id < 10 +) +SELECT * FROM z; + +-- multiple with works 5 +WITH RECURSIVE x AS ( + SELECT 1 AS id + UNION ALL + SELECT id + 1 FROM x WHERE id < 3 +), +y AS ( + SELECT * FROM x + UNION ALL + SELECT * FROM x +), +z AS ( + SELECT * FROM y + UNION ALL + SELECT id + 1 FROM z WHERE id < 10 +) +SELECT * FROM z; + +-- recursion nested into WITH +WITH t AS ( + WITH RECURSIVE s AS ( + VALUES (1) AS t(i) + UNION ALL + SELECT i + 1 FROM s + ) + SELECT i AS j FROM s LIMIT 10 +) +SELECT * FROM t; + +-- WITH nested into recursion +WITH RECURSIVE outermost AS ( + SELECT 0 AS level + UNION ALL + (WITH innermost AS ( + SELECT * FROM outermost + ) + SELECT level + 1 FROM innermost WHERE level < 5) +) +SELECT * FROM outermost; + +-- recursion nested into recursion +WITH RECURSIVE t AS ( + WITH RECURSIVE s AS ( + VALUES (1) AS t(i) + UNION ALL + SELECT i + 1 FROM s WHERE i < 10 + ) + SELECT i AS j FROM s + UNION ALL + SELECT j + 1 FROM t WHERE j < 10 +) +SELECT * FROM t; + +-- recursion nested into recursion 2 +WITH RECURSIVE t AS ( + WITH RECURSIVE s AS ( + SELECT j, 1 AS i FROM t + UNION ALL + SELECT j, i + 1 FROM s WHERE i < 3 + ) + VALUES (1) as t(j) + UNION ALL + SELECT j + 1 FROM s WHERE j < 3 +) +SELECT * FROM t; + +-- name collision of nested recursions +WITH RECURSIVE r(level) AS ( + WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 + ) + SELECT * FROM r + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 +) +SELECT * FROM r; + +-- name collision of nested recursions 2 +WITH RECURSIVE r(level) AS ( + (WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 + ) + SELECT * FROM r) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 +) +SELECT * FROM r; + +-- exchange reuse with recursion +WITH RECURSIVE r(level, id) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, t.id + FROM r + LEFT JOIN t ON t.id = r.level + WHERE level < 10 +) +SELECT * +FROM r AS r1 +JOIN r AS r2 ON r1.id = r2.id + 1 AND r1.level = r2.level + 1; + +-- 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 IF EXISTS 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; + +-- Clean up +DROP VIEW IF EXISTS t; 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 80bf258704c70..fcf22f2218d16 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/explain.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/explain.sql @@ -113,6 +113,14 @@ EXPLAIN FORMATTED FROM explain_temp4 GROUP BY key; +EXPLAIN FORMATTED + WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 + ) + SELECT * FROM r; + -- cleanup DROP TABLE explain_temp1; DROP TABLE explain_temp2; 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 83c6fd8cbac91..7118036e61903 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 @@ -26,7 +26,7 @@ SELECT * FROM q1, q1 AS q2; -- SELECT * FROM q1 --) ss; SELECT count(*) FROM ( - WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + WITH q1(x) AS (SELECT random() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) SELECT * FROM q1 UNION SELECT * FROM q1 @@ -35,49 +35,62 @@ 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; --- [SPARK-24497] Support recursive SQL query +SELECT * FROM nums; + +-- [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 +-- [SPARK-28731] Support limit on recursive queries --WITH RECURSIVE t(n) AS ( -- VALUES (1) --UNION ALL @@ -85,7 +98,7 @@ SELECT count(*) FROM ( --SELECT * FROM t LIMIT 10; -- UNION case should have same property --- [SPARK-24497] Support recursive SQL query +-- [SPARK-28731] Support limit on recursive queries --WITH RECURSIVE t(n) AS ( -- SELECT 1 --UNION @@ -94,28 +107,43 @@ SELECT count(*) FROM ( -- 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 @@ -127,7 +155,6 @@ SELECT count(*) FROM ( -- | +->D-+->F -- +->E-+->G - -- [ORIGINAL SQL] --CREATE TEMP TABLE department ( -- id INTEGER PRIMARY KEY, -- department ID @@ -151,62 +178,59 @@ 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 +-- [NOTE] Decreased recursion depth to avoid stack overflow +-- [ORIGINAL SQL] --SELECT count(*) FROM ( -- WITH RECURSIVE t(n) AS ( -- SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 500 @@ -218,16 +242,28 @@ INSERT INTO department VALUES (7, 5, 'G'); -- ) -- SELECT * FROM t WHERE n < 50000 -- ) AS t WHERE n < 100); +SET spark.sql.cte.recursion.level.limit=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.cte.recursion.level.limit=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 +-- [ORIGINAL SQL] --CREATE TEMPORARY VIEW vsubdepartment AS -- WITH RECURSIVE subdepartment AS -- ( @@ -241,25 +277,45 @@ INSERT INTO department VALUES (7, 5, 'G'); -- SELECT * FROM subdepartment; -- --SELECT * FROM vsubdepartment ORDER BY name; --- +CREATE 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 +-- [ORIGINAL SQL] --SELECT pg_get_viewdef('vsubdepartment'::regclass); --SELECT pg_get_viewdef('vsubdepartment'::regclass, true); +SHOW CREATE TABLE vsubdepartment AS SERDE; + +DROP VIEW vsubdepartment; -- 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; --- +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; + +-- [ORIGINAL SQL] --\d+ sums_1_100 +SHOW CREATE TABLE sums_1_100 AS SERDE; + +DROP VIEW sums_1_100; -- corner case in which sub-WITH gets initialized first --- [SPARK-24497] Support recursive SQL query +-- [SPARK-28731] Support limit on recursive queries --with recursive q as ( -- select * from department -- union all @@ -268,7 +324,7 @@ INSERT INTO department VALUES (7, 5, 'G'); -- ) --select * from q limit 24; --- [SPARK-24497] Support recursive SQL query +-- [SPARK-28731] Support limit on recursive queries --with recursive q as ( -- select * from department -- union all @@ -282,15 +338,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 +367,10 @@ 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 +-- [SPARK-28400] Add built-in Array Functions: array_upper +-- [SPARK-28402] Array indexing 1-based in PostgreSQL +-- [SPARK-28401] Add built-in Array Functions: element to array concatenation +-- [ORIGINAL SQL] --WITH RECURSIVE t(id, path) AS ( -- VALUES(1,ARRAY[]::integer[]) --UNION ALL @@ -324,9 +382,23 @@ 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 +-- [SPARK-28400] Add built-in Array Functions: array_upper +-- [SPARK-28402] Array indexing 1-based in PostgreSQL +-- [SPARK-28401] Add built-in Array Functions: element to array concatenation +-- [ORIGINAL SQL] --WITH RECURSIVE t(id, path) AS ( -- VALUES(1,ARRAY[]::integer[]) --UNION ALL @@ -339,9 +411,23 @@ 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(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) + 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 +-- [SPARK-28401] Add built-in Array Functions: element to array concatenation +-- [NOTE] Spark SQL doesn't support Record Type, so Struct Type is used instead +-- [ORIGINAL SQL] --WITH RECURSIVE t(id, path) AS ( -- VALUES(1,ARRAY[]::integer[]) --UNION ALL @@ -350,6 +436,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 +460,9 @@ insert into graph values (4, 5, 'arc 4 -> 5'), (5, 1, 'arc 5 -> 1'); --- [SPARK-24497] Support recursive SQL query +-- [SPARK-26854] Support ANY/SOME subquery +-- [NOTE] Spark SQL doesn't support Record Type, so Struct Type is used instead +-- [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 +471,19 @@ 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 +-- [SPARK-26854] Support ANY/SOME subquery +-- [NOTE] Spark SQL doesn't support Record Type, so Struct Type is used instead +-- [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,58 +492,76 @@ 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 ---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; +-- [NOTE] Spark SQL doesn't support forward references in WITH RECURSIVE clauses +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 +-- [ORIGINAL SQL] --WITH RECURSIVE -- x(id) AS -- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), -- y(id) AS -- (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 +WITH RECURSIVE + x(id) AS + (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), + y(id) AS + (VALUES (1) UNION ALL SELECT id+1 FROM y WHERE id < 10) + SELECT y.*, x.* FROM y LEFT JOIN x ON x.id = y.id; + +-- [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), -- y(id) AS -- (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 + (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), + y(id) AS + (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 10) + SELECT y.*, x.* FROM y LEFT JOIN x ON x.id = y.id; + +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 +608,26 @@ DROP TABLE y; -- -- INTERSECT --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x) + SELECT * FROM x; --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x) + SELECT * FROM x; -- EXCEPT --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x) + SELECT * FROM x; --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x) + SELECT * FROM x; -- no non-recursive term --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT n FROM x) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT n FROM x) + SELECT * FROM x; -- recursive term in the left hand side (strictly speaking, should allow this) --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1) --- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1) + SELECT * FROM x; -- [ORIGINAL SQL] --CREATE TEMPORARY TABLE y (a INTEGER); @@ -520,123 +638,131 @@ 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 +-- [NOTE] Spark SQL doesn't handle non-matching types in recursion +-- [ORIGINAL SQL] --WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT count(*) FROM x) -- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1L UNION ALL SELECT count(*) FROM x) + SELECT * FROM x; --- [SPARK-24497] Support recursive SQL query +-- [NOTE] Spark SQL doesn't handle non-matching types in recursion +-- [ORIGINAL SQL] --WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT sum(n) FROM x) -- SELECT * FROM x; +WITH RECURSIVE x(n) AS (SELECT 1L 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 +-- [SPARK-28330] Enhance query limit +-- [ORIGINAL SQL] --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) + SELECT * FROM x; -- FOR UPDATE --- [SPARK-24497] Support recursive SQL query +-- [NOTE] Spark SQL doesn't support FOR UPDATE clause --WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x FOR UPDATE) -- SELECT * FROM x; -- target list has a recursive query name --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE x(id) AS (values (1) --- UNION ALL --- SELECT (SELECT * FROM x) FROM x WHERE id < 5 ---) SELECT * FROM x; +WITH RECURSIVE x(id) AS (values (1) + UNION ALL + SELECT (SELECT * FROM x) FROM x WHERE id < 5 +) SELECT * FROM x; -- mutual recursive query (not implemented) --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE --- x (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM y WHERE id < 5), --- y (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 5) ---SELECT * FROM x; +WITH RECURSIVE + x (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM y WHERE id < 5), + y (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 5) +SELECT * FROM x; -- non-linear recursion is not allowed --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE foo(i) AS --- (values (1) --- UNION ALL --- (SELECT i+1 FROM foo WHERE i < 10 --- UNION ALL --- SELECT i+1 FROM foo WHERE i < 5) ---) SELECT * FROM foo; - --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE foo(i) AS --- (values (1) --- UNION ALL --- SELECT * FROM --- (SELECT i+1 FROM foo WHERE i < 10 --- UNION ALL --- SELECT i+1 FROM foo WHERE i < 5) AS t ---) SELECT * FROM foo; - --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE foo(i) AS --- (values (1) --- UNION ALL --- (SELECT i+1 FROM foo WHERE i < 10 --- EXCEPT --- SELECT i+1 FROM foo WHERE i < 5) ---) SELECT * FROM foo; - --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE foo(i) AS --- (values (1) --- UNION ALL --- (SELECT i+1 FROM foo WHERE i < 10 --- INTERSECT --- SELECT i+1 FROM foo WHERE i < 5) ---) SELECT * FROM foo; +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + UNION ALL + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo; + +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + SELECT * FROM + (SELECT i+1 FROM foo WHERE i < 10 + UNION ALL + SELECT i+1 FROM foo WHERE i < 5) AS t +) SELECT * FROM foo; + +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + EXCEPT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo; + +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + INTERSECT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo; -- Wrong type induced from non-recursive term --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --WITH RECURSIVE foo(i) AS -- (SELECT i FROM (VALUES(1),(2)) t(i) -- UNION ALL -- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) --SELECT * FROM foo; +WITH RECURSIVE foo(i) AS + (SELECT i FROM (VALUES(1),(2)) t(i) + UNION ALL + SELECT cast((i+1) AS decimal(10,0)) FROM foo WHERE i < 10) +SELECT * FROM foo; -- rejects different typmod, too (should we allow this?) --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --WITH RECURSIVE foo(i) AS -- (SELECT i::numeric(3,0) FROM (VALUES(1),(2)) t(i) -- UNION ALL -- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) --SELECT * FROM foo; +WITH RECURSIVE foo(i) AS + (SELECT cast(i as decimal(3,0)) FROM (VALUES(1),(2)) t(i) + UNION ALL + SELECT cast((i+1) AS decimal(10,0)) FROM foo WHERE i < 10) +SELECT * FROM foo; -- [NOTE] Spark SQL doesn't support RULEs -- disallow OLD/NEW reference in CTE @@ -668,18 +794,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 +826,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,7 +860,7 @@ SELECT * FROM outermost ORDER BY 1; -- Test CTEs read in non-initialization orders -- --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --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 ( @@ -763,8 +886,33 @@ SELECT * FROM outermost ORDER BY 1; -- ) -- ) --SELECT * FROM iter; +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 ( + SELECT 0, 'base', 17 + UNION ALL ( + WITH remaining(id_key, row_type, link, min) AS ( + SELECT tab.id_key, 'true', iter.link, MIN(tab.id_key) OVER () + FROM tab INNER JOIN iter ON iter.link=tab.link + WHERE tab.id_key > iter.id_key + ), + first_remaining AS ( + SELECT id_key, row_type, link + FROM remaining + WHERE id_key=min + ), + effect AS ( + SELECT tab.id_key, 'new', tab.link + FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key + WHERE e.row_type = 'false' + ) + SELECT * FROM first_remaining + UNION ALL SELECT * FROM effect + ) + ) +SELECT * FROM iter; --- [SPARK-24497] Support recursive SQL query +-- [ORIGINAL SQL] --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 ( @@ -790,6 +938,31 @@ SELECT * FROM outermost ORDER BY 1; -- ) -- ) --SELECT * FROM iter; +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 ( + SELECT 0, 'base', 17 + UNION ( + WITH remaining(id_key, row_type, link, min) AS ( + SELECT tab.id_key, 'true', iter.link, MIN(tab.id_key) OVER () + FROM tab INNER JOIN iter ON iter.link=tab.link + WHERE tab.id_key > iter.id_key + ), + first_remaining AS ( + SELECT id_key, row_type, link + FROM remaining + WHERE id_key=min + ), + effect AS ( + SELECT tab.id_key, 'new', tab.link + FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key + WHERE e.row_type = 'false' + ) + SELECT * FROM first_remaining + UNION ALL SELECT * FROM effect + ) + ) +SELECT * FROM iter; -- -- Data-modifying statements in WITH @@ -895,7 +1068,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 +-- [SPARK-28147] Support RETURNING clause --WITH RECURSIVE t(a) AS ( -- SELECT 0 -- UNION ALL @@ -1003,7 +1176,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 +1187,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,19 +1319,17 @@ SELECT * FROM parent; -- error cases -- data-modifying WITH tries to use its own output --- [SPARK-24497] Support recursive SQL query ---WITH RECURSIVE t AS ( --- INSERT INTO y --- SELECT * FROM t ---) ---VALUES(FALSE); +WITH RECURSIVE t AS ( + INSERT INTO y + SELECT * FROM t +) +VALUES(FALSE); -- no RETURNING in a referenced data-modifying WITH --- [SPARK-24497] Support recursive SQL query ---WITH t AS ( --- INSERT INTO y VALUES(0) ---) ---SELECT * FROM t; +WITH t AS ( + INSERT INTO y VALUES(0) +) +SELECT * FROM t; -- data-modifying WITH allowed only at the top level -- [SPARK-28147] Support RETURNING clause 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..97db69f841fa2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out @@ -0,0 +1,935 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 42 + + +-- !query +CREATE TEMPORARY VIEW t AS SELECT * FROM VALUES 0, 1, 2 AS t(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Table or view not found: r; line 4 pos 24 + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SET spark.sql.cte.recursion.level.limit=100 +-- !query schema +struct +-- !query output +spark.sql.cte.recursion.level.limit 100 + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cte.recursion.level.limit + + +-- !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 +org.apache.spark.SparkException +Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cte.recursion.level.limit + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r +) +SELECT level, level FROM r LIMIT 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cte.recursion.level.limit + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r +) +SELECT level, level FROM r ORDER BY level LIMIT 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cte.recursion.level.limit + + +-- !query +WITH RECURSIVE r(c) AS ( + SELECT 'a' + UNION ALL + SELECT c || ' b' FROM r WHERE LENGTH(c) < 10 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +a +a b +a b b +a b b b +a b b b b +a b b b b b + + +-- !query +WITH RECURSIVE r(level) AS ( + SELECT level + 1 FROM r WHERE level < 10 + UNION ALL + VALUES (0) +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive query r should not contain recursive references in its anchor (first) term.; + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + VALUES (0, 'B') + UNION ALL + SELECT level + 1, data || 'C' FROM r WHERE level < 3 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 A +0 B +1 AC +1 BC +2 ACC +2 BCC +3 ACCC +3 BCCC + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT level + 1, data || 'B' FROM r WHERE level < 2 + UNION ALL + SELECT level + 1, data || 'C' FROM r WHERE level < 3 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive query r should not contain recursive references in its anchor (first) term.; + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + VALUES (0, 'B') + UNION ALL + SELECT level + 1, data || 'C' FROM r WHERE level < 2 + UNION ALL + SELECT level + 1, data || 'D' FROM r WHERE level < 3 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive query r should not contain recursive references in its anchor (first) term.; + + +-- !query +WITH RECURSIVE r(level) AS ( + SELECT level + 1 FROM r WHERE level < 3 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive query r should contain UNION or UNION ALL statements only. This error can also be caused by ORDER BY or LIMIT keywords used on result of UNION or UNION ALL.; + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0), (0) + UNION + SELECT (level + 1) % 10 FROM r +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0) + INTERSECT + SELECT level + 1 FROM r WHERE level < 10 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive query r should contain UNION or UNION ALL statements only. This error can also be caused by ORDER BY or LIMIT keywords used on result of UNION or UNION ALL.; + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE (SELECT SUM(level) FROM r) < 10 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive query r should not contain recursive references in its subquery.; + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT r1.level + 1, r1.data + FROM r AS r1 + JOIN r AS r2 ON r2.data = r1.data + WHERE r1.level < 10 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive reference r cannot be used multiple times in a recursive term.; + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT level + 1, r.data + FROM ( + SELECT 'B' AS data + ) AS o + LEFT JOIN r ON r.data = o.data +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive reference r cannot be used here. This can be caused by using it on inner side of an outer join, using it with aggregate in a subquery or using it multiple times in a recursive term (except for using it on different sides of an UNION ALL).; + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT level + 1, r.data + FROM r + RIGHT JOIN ( + SELECT 'B' AS data + ) AS o ON o.data = r.data +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive reference r cannot be used here. This can be caused by using it on inner side of an outer join, using it with aggregate in a subquery or using it multiple times in a recursive term (except for using it on different sides of an UNION ALL).; + + +-- !query +WITH RECURSIVE r(level, data) AS ( + SELECT MAX(level) AS level, SUM(data) AS data FROM VALUES (0, 1), (0, 2) + UNION ALL + SELECT level + 1, data FROM r WHERE level < 10 +) +SELECT * FROM r ORDER BY level +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve '`level`' given input columns: [col1, col2]; line 2 pos 13 + + +-- !query +WITH RECURSIVE r(id, data) AS ( + VALUES (0, 1L) + UNION ALL + SELECT 1, SUM(data) FROM r WHERE data < 10 GROUP BY id +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive reference r cannot be used here. This can be caused by using it on inner side of an outer join, using it with aggregate in a subquery or using it multiple times in a recursive term (except for using it on different sides of an UNION ALL).; + + +-- !query +WITH RECURSIVE r(level) AS ( + VALUES (1L) + UNION ALL + SELECT SUM(level) FROM r WHERE level < 10 +) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive reference r cannot be used here. This can be caused by using it on inner side of an outer join, using it with aggregate in a subquery or using it multiple times in a recursive term (except for using it on different sides of an UNION ALL).; + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT level + 1, data FROM r WHERE level < 10 +) +SELECT COUNT(*) FROM r +-- !query schema +struct +-- !query output +11 + + +-- !query +WITH RECURSIVE r(level, data) AS ( + VALUES (0, 'A') + UNION ALL + SELECT DISTINCT level + 1, data FROM r WHERE level < 10 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 A +1 A +10 A +2 A +3 A +4 A +5 A +6 A +7 A +8 A +9 A + + +-- !query +WITH RECURSIVE y AS ( + VALUES (1) AS t(id) +), +x 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 AS ( + VALUES (1) AS t(id) + UNION ALL + SELECT id + 1 FROM x WHERE id < 5 +), +y AS ( + VALUES (1) AS t(id) + UNION ALL + SELECT id + 1 FROM y WHERE id < 10 +) +SELECT * FROM y LEFT JOIN x ON x.id = y.id +-- !query schema +struct +-- !query output +1 1 +10 NULL +2 2 +3 3 +4 4 +5 5 +6 NULL +7 NULL +8 NULL +9 NULL + + +-- !query +WITH RECURSIVE x AS ( + VALUES (1) AS t(id) + UNION ALL + SELECT id + 1 FROM x WHERE id < 5 +), +y AS ( + VALUES (1) AS t(id) + UNION ALL + SELECT id + 1 FROM x WHERE id < 10 +) +SELECT * FROM y LEFT JOIN x ON x.id = y.id +-- !query schema +struct +-- !query output +1 1 +2 2 +3 3 +4 4 +5 5 +6 NULL + + +-- !query +WITH RECURSIVE x AS ( + SELECT 1 AS id + UNION ALL + SELECT id + 1 FROM x WHERE id < 3 +), +y AS ( + SELECT * FROM x + UNION ALL + SELECT * FROM x +), +z 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 AS ( + SELECT 1 AS id + UNION ALL + SELECT id + 1 FROM x WHERE id < 3 +), +y AS ( + SELECT * FROM x + UNION ALL + SELECT * FROM x +), +z 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 +WITH t AS ( + WITH RECURSIVE s AS ( + VALUES (1) AS t(i) + UNION ALL + SELECT i + 1 FROM s + ) + SELECT i AS j FROM s LIMIT 10 +) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cte.recursion.level.limit + + +-- !query +WITH RECURSIVE outermost AS ( + SELECT 0 AS level + UNION ALL + (WITH innermost AS ( + SELECT * FROM outermost + ) + SELECT level + 1 FROM innermost WHERE level < 5) +) +SELECT * FROM outermost +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 + + +-- !query +WITH RECURSIVE t AS ( + WITH RECURSIVE s AS ( + VALUES (1) AS t(i) + UNION ALL + SELECT i + 1 FROM s WHERE i < 10 + ) + SELECT i AS j 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 RECURSIVE t AS ( + WITH RECURSIVE s AS ( + SELECT j, 1 AS i FROM t + UNION ALL + SELECT j, i + 1 FROM s WHERE i < 3 + ) + VALUES (1) as t(j) + UNION ALL + SELECT j + 1 FROM s WHERE j < 3 +) +SELECT * FROM t +-- !query schema +struct +-- !query output +1 +2 +2 +2 +3 +3 +3 +3 +3 +3 +3 +3 +3 + + +-- !query +WITH RECURSIVE r(level) AS ( + WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 + ) + SELECT * FROM r + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 +1 +1 +10 +10 +2 +2 +3 +3 +4 +4 +5 +5 +6 +6 +7 +7 +8 +8 +9 +9 + + +-- !query +WITH RECURSIVE r(level) AS ( + (WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 + ) + SELECT * FROM r) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 +) +SELECT * FROM r +-- !query schema +struct +-- !query output +0 +1 +1 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +2 +2 +2 +3 +3 +3 +3 +4 +4 +4 +4 +4 +5 +5 +5 +5 +5 +5 +6 +6 +6 +6 +6 +6 +6 +7 +7 +7 +7 +7 +7 +7 +7 +8 +8 +8 +8 +8 +8 +8 +8 +8 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 + + +-- !query +WITH RECURSIVE r(level, id) AS ( + VALUES (0, 0) + UNION ALL + SELECT level + 1, t.id + FROM r + LEFT JOIN t ON t.id = r.level + WHERE level < 10 +) +SELECT * +FROM r AS r1 +JOIN r AS r2 ON r1.id = r2.id + 1 AND r1.level = r2.level + 1 +-- !query schema +struct +-- !query output +2 1 1 0 +3 2 2 1 + + +-- !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 IF EXISTS 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 +DROP VIEW IF EXISTS t +-- !query schema +struct<> +-- !query output + 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 36757863ffcb5..48f808ddc2f70 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 23 +-- Number of queries: 24 -- !query @@ -828,6 +828,47 @@ Output [2]: [key#x, min(val)#x] Arguments: isFinalPlan=false +-- !query +EXPLAIN FORMATTED + WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 + ) + SELECT * FROM r +-- !query schema +struct +-- !query output +== Physical Plan == +RecursiveRelation (3) +: +- Project (6) +: +- Filter (5) +: +- RecursiveReference (4) ++- * Project (2) + +- * LocalTableScan (1) + + +(1) LocalTableScan [codegen id : 1] +Output [1]: [col1#x] +Arguments: [col1#x] + +(2) Project [codegen id : 1] +Output [1]: [col1#x AS level#x] +Input [1]: [col1#x] + +(3) RecursiveRelation +Arguments: r, [level#x] + +(4) RecursiveReference +Arguments: r, [level#x], false, 0 + +(5) Filter +Arguments: (level#x < 10) + +(6) Project +Arguments: [(level#x + 1) AS (level + 1)#x] + + -- !query DROP TABLE explain_temp1 -- !query schema 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 2b07dac0e5d0a..626979b8023df 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 23 +-- Number of queries: 24 -- !query @@ -1031,6 +1031,47 @@ Aggregate Attributes [1]: [min(val#x)#x] Results [2]: [key#x, min(val#x)#x AS min(val)#x] +-- !query +EXPLAIN FORMATTED + WITH RECURSIVE r(level) AS ( + VALUES (0) + UNION ALL + SELECT level + 1 FROM r WHERE level < 10 + ) + SELECT * FROM r +-- !query schema +struct +-- !query output +== Physical Plan == +RecursiveRelation (3) +: +- Project (6) +: +- Filter (5) +: +- RecursiveReference (4) ++- * Project (2) + +- * LocalTableScan (1) + + +(1) LocalTableScan [codegen id : 1] +Output [1]: [col1#x] +Arguments: [col1#x] + +(2) Project [codegen id : 1] +Output [1]: [col1#x AS level#x] +Input [1]: [col1#x] + +(3) RecursiveRelation +Arguments: r, [level#x] + +(4) RecursiveReference +Arguments: r, [level#x], false, 0 + +(5) Filter +Arguments: (level#x < 10) + +(6) Project +Arguments: [(level#x + 1) AS (level + 1)#x] + + -- !query DROP TABLE explain_temp1 -- !query schema 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 badafc9e659e2..db47f33b31e1b 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 51 +-- Number of queries: 117 -- !query @@ -13,7 +13,7 @@ struct -- !query SELECT count(*) FROM ( - WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + WITH q1(x) AS (SELECT random() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) SELECT * FROM q1 UNION SELECT * FROM q1 @@ -24,6 +24,143 @@ struct 10 +-- !query +WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t WHERE n < 100 +) +SELECT sum(n) FROM t +-- !query schema +struct +-- !query output +5050 + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT (VALUES(1)) +UNION ALL + SELECT n+1 FROM t WHERE n < 5 +) +SELECT * FROM t +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 + + +-- !query +CREATE TEMPORARY VIEW nums AS +WITH RECURSIVE nums (n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM nums WHERE n < 5 +) +SELECT * FROM nums +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM nums +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW nums AS +WITH RECURSIVE nums (n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM nums WHERE n < 6 +) +SELECT * FROM nums +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM nums +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 +6 + + +-- !query +WITH RECURSIVE t(n) AS ( + SELECT 1 +UNION + SELECT 10-n FROM t) +SELECT * FROM t +-- !query schema +struct +-- !query output +1 +9 + + +-- !query +WITH 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 +Failed to merge incompatible data types string and double + + -- !query CREATE TABLE department ( id INTEGER, -- department ID @@ -31,13 +168,682 @@ 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.cte.recursion.level.limit=200 +-- !query schema +struct +-- !query output +spark.sql.cte.recursion.level.limit 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.cte.recursion.level.limit=100 +-- !query schema +struct +-- !query output +spark.sql.cte.recursion.level.limit 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 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 +SHOW CREATE TABLE vsubdepartment AS SERDE +-- !query schema +struct +-- !query output +CREATE VIEW `default`.`vsubdepartment`( + `id`, + `parent_department`, + `name`) +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 +DROP VIEW vsubdepartment +-- !query schema +struct<> +-- !query output + + + +-- !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 +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE sums_1_100 AS SERDE +-- !query schema +struct +-- !query output +CREATE VIEW `default`.`sums_1_100`( + `sum(n)`) +AS WITH RECURSIVE t(n) AS ( + VALUES (1) +UNION ALL + SELECT n+1 FROM t WHERE n < 100 +) +SELECT sum(n) FROM t + + +-- !query +DROP VIEW sums_1_100 +-- !query schema +struct<> +-- !query output + + + +-- !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(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) + 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 * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5), + y(id) AS (values (1)) + SELECT * FROM x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Table or view not found: y; line 2 pos 28 + + +-- !query +WITH RECURSIVE + x(id) AS + (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), + y(id) AS + (VALUES (1) UNION ALL SELECT id+1 FROM y WHERE id < 10) + SELECT y.*, x.* FROM y LEFT JOIN x ON x.id = y.id +-- !query schema +struct +-- !query output +1 1 +10 NULL +2 2 +3 3 +4 4 +5 5 +6 NULL +7 NULL +8 NULL +9 NULL + + +-- !query +WITH RECURSIVE + x(id) AS + (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), + y(id) AS + (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 10) + SELECT y.*, x.* FROM y LEFT JOIN x ON x.id = y.id +-- !query schema +struct +-- !query output +1 1 +2 2 +3 3 +4 4 +5 5 +6 NULL + + +-- !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 @@ -45,7 +851,7 @@ struct<> -- !query -INSERT INTO department VALUES (1, 0, 'A') +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) -- !query schema struct<> -- !query output @@ -53,7 +859,7 @@ struct<> -- !query -INSERT INTO department VALUES (2, 1, 'B') +DROP TABLE y -- !query schema struct<> -- !query output @@ -61,39 +867,67 @@ 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 +Recursive query x should contain UNION or UNION ALL statements only. This error can also be caused by ORDER BY or LIMIT keywords used on result of UNION or UNION ALL.; -- !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 +Recursive query x should contain UNION or UNION ALL statements only. This error can also be caused by ORDER BY or LIMIT keywords used on result of UNION or UNION ALL.; -- !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 +Recursive query x should contain UNION or UNION ALL statements only. This error can also be caused by ORDER BY or LIMIT keywords used on result of UNION or UNION ALL.; + +-- !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 +Recursive query x should contain UNION or UNION ALL statements only. This error can also be caused by ORDER BY or LIMIT keywords used on result of UNION or UNION ALL.; -- !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 +Recursive query x should contain UNION or UNION ALL statements only. This error can also be caused by ORDER BY or LIMIT keywords used on result of UNION or UNION ALL.; + +-- !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.AnalysisException +Recursive query x should not contain recursive references in its anchor (first) term.; -- !query -INSERT INTO department VALUES (7, 5, 'G') +CREATE TABLE y (a INTEGER) USING parquet -- !query schema struct<> -- !query output @@ -101,10 +935,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 @@ -112,75 +943,419 @@ 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.AnalysisException +Recursive reference x cannot be used here. This can be caused by using it on inner side of an outer join, using it with aggregate in a subquery or using it multiple times in a recursive term (except for using it on different sides of an UNION ALL).; -- !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.AnalysisException +Recursive reference x cannot be used here. This can be caused by using it on inner side of an outer join, using it with aggregate in a subquery or using it multiple times in a recursive term (except for using it on different sides of an UNION ALL).; + +-- !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.AnalysisException +Recursive reference x cannot be used here. This can be caused by using it on inner side of an outer join, using it with aggregate in a subquery or using it multiple times in a recursive term (except for using it on different sides of an UNION ALL).; -- !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 +Recursive query x should not contain recursive references in its subquery.; + +-- !query +WITH RECURSIVE x(n) AS (SELECT 1L UNION ALL SELECT count(*) FROM x) + SELECT * FROM x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive reference x cannot be used here. This can be caused by using it on inner side of an outer join, using it with aggregate in a subquery or using it multiple times in a recursive term (except for using it on different sides of an UNION ALL).; -- !query -CREATE TABLE y (a INTEGER) USING parquet +WITH RECURSIVE x(n) AS (SELECT 1L UNION ALL SELECT sum(n) FROM x) + SELECT * FROM x -- !query schema struct<> -- !query output +org.apache.spark.sql.AnalysisException +Recursive reference x cannot be used here. This can be caused by using it on inner side of an outer join, using it with aggregate in a subquery or using it multiple times in a recursive term (except for using it on different sides of an UNION ALL).; + +-- !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 +Recursive query x should contain UNION or UNION ALL statements only. This error can also be caused by ORDER BY or LIMIT keywords used on result of UNION or UNION ALL.; -- !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) + SELECT * FROM x -- !query schema struct<> -- !query output +org.apache.spark.sql.AnalysisException +Recursive query x should contain UNION or UNION ALL statements only. This error can also be caused by ORDER BY or LIMIT keywords used on result of UNION or UNION ALL.; + +-- !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 +Recursive query x should not contain recursive references in its subquery.; -- !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.AnalysisException +cannot resolve '`id`' given input columns: [spark_catalog.default.y.a]; line 2 pos 57 + + +-- !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 +1 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +2 +2 +3 +3 +3 +3 +4 +4 +4 +4 +4 +4 +4 +4 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +-- !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 schema +struct +-- !query output +1 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +2 +2 +3 +3 +3 +3 +4 +4 +4 +4 +4 +4 +4 +4 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +6 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +7 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +8 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 +9 + -- !query -CREATE TABLE y (a INTEGER) USING parquet +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + EXCEPT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo -- !query schema struct<> -- !query output +org.apache.spark.sql.AnalysisException +Recursive reference foo cannot be used multiple times in a recursive term.; +-- !query +WITH RECURSIVE foo(i) AS + (values (1) + UNION ALL + (SELECT i+1 FROM foo WHERE i < 10 + INTERSECT + SELECT i+1 FROM foo WHERE i < 5) +) SELECT * FROM foo +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive reference foo cannot be used multiple times in a recursive term.; + -- !query -INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +WITH RECURSIVE foo(i) AS + (SELECT i FROM (VALUES(1),(2)) t(i) + UNION ALL + SELECT cast((i+1) AS decimal(10,0)) FROM foo WHERE i < 10) +SELECT * FROM foo -- !query schema struct<> -- !query output +org.apache.spark.SparkException +Failed to merge incompatible data types int and decimal(10,0) + +-- !query +WITH RECURSIVE foo(i) AS + (SELECT cast(i as decimal(3,0)) FROM (VALUES(1),(2)) t(i) + UNION ALL + SELECT cast((i+1) AS decimal(10,0)) FROM foo WHERE i < 10) +SELECT * FROM foo +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Failed to merge decimal types with incompatible precision 3 and 10 -- !query @@ -191,6 +1366,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 @@ -222,6 +1469,111 @@ org.apache.spark.sql.AnalysisException Table or view not found: outermost; line 4 pos 23 +-- !query +WITH RECURSIVE outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1 +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +WITH RECURSIVE outermost(x) AS ( + WITH innermost as (SELECT 2 FROM outermost) -- fail + SELECT * FROM innermost + UNION SELECT * from outermost +) +SELECT * FROM outermost ORDER BY 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Recursive query outermost should not contain recursive references in its anchor (first) term.; + + +-- !query +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 ( + SELECT 0, 'base', 17 + UNION ALL ( + WITH remaining(id_key, row_type, link, min) AS ( + SELECT tab.id_key, 'true', iter.link, MIN(tab.id_key) OVER () + FROM tab INNER JOIN iter ON iter.link=tab.link + WHERE tab.id_key > iter.id_key + ), + first_remaining AS ( + SELECT id_key, row_type, link + FROM remaining + WHERE id_key=min + ), + effect AS ( + SELECT tab.id_key, 'new', tab.link + FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key + WHERE e.row_type = 'false' + ) + SELECT * FROM first_remaining + UNION ALL SELECT * FROM effect + ) + ) +SELECT * FROM iter +-- !query schema +struct +-- !query output +0 base 17 +1 true 17 +2 true 17 +3 true 17 +4 true 17 +5 true 17 +6 true 17 + + +-- !query +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 ( + SELECT 0, 'base', 17 + UNION ( + WITH remaining(id_key, row_type, link, min) AS ( + SELECT tab.id_key, 'true', iter.link, MIN(tab.id_key) OVER () + FROM tab INNER JOIN iter ON iter.link=tab.link + WHERE tab.id_key > iter.id_key + ), + first_remaining AS ( + SELECT id_key, row_type, link + FROM remaining + WHERE id_key=min + ), + effect AS ( + SELECT tab.id_key, 'new', tab.link + FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key + WHERE e.row_type = 'false' + ) + SELECT * FROM first_remaining + UNION ALL SELECT * FROM effect + ) + ) +SELECT * FROM iter +-- !query schema +struct +-- !query output +0 base 17 +1 true 17 +2 true 17 +3 true 17 +4 true 17 +5 true 17 +6 true 17 + + -- !query CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i || ' v' AS string) v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i) -- !query schema @@ -343,6 +1695,48 @@ struct 1 p1 +-- !query +WITH RECURSIVE t AS ( + INSERT INTO y + SELECT * FROM t +) +VALUES(FALSE) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'WITH RECURSIVE t AS (\n\tINSERT'(line 2, pos 1) + +== SQL == +WITH RECURSIVE t AS ( + INSERT INTO y +-^^^ + SELECT * FROM t +) +VALUES(FALSE) + + +-- !query +WITH t AS ( + INSERT INTO y VALUES(0) +) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'WITH t AS (\n\tINSERT'(line 2, pos 1) + +== SQL == +WITH t AS ( + INSERT INTO y VALUES(0) +-^^^ +) +SELECT * FROM t + + -- !query create table foo (with baz) -- !query schema