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 d75e7d528d5b3..84f6d04eeb1a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -123,7 +123,7 @@ object CTESubstitution extends Rule[LogicalPlan] { startOfQuery: Boolean = true): Unit = { val resolver = conf.resolver plan match { - case UnresolvedWith(child, relations) => + case UnresolvedWith(child, relations, _) => val newNames = ArrayBuffer.empty[String] newNames ++= outerCTERelationNames relations.foreach { @@ -149,7 +149,7 @@ object CTESubstitution extends Rule[LogicalPlan] { plan: LogicalPlan, cteDefs: ArrayBuffer[CTERelationDef]): LogicalPlan = { plan.resolveOperatorsUp { - case UnresolvedWith(child, relations) => + case UnresolvedWith(child, relations, _) => val resolvedCTERelations = resolveCTERelations(relations, isLegacy = true, forceInline = false, Seq.empty, cteDefs) substituteCTE(child, alwaysInline = true, resolvedCTERelations) @@ -202,7 +202,7 @@ object CTESubstitution extends Rule[LogicalPlan] { var firstSubstituted: Option[LogicalPlan] = None val newPlan = plan.resolveOperatorsDownWithPruning( _.containsAnyPattern(UNRESOLVED_WITH, PLAN_EXPRESSION)) { - case UnresolvedWith(child: LogicalPlan, relations) => + case UnresolvedWith(child: LogicalPlan, relations, _) => val resolvedCTERelations = resolveCTERelations(relations, isLegacy = false, forceInline, outerCTEDefs, cteDefs) ++ outerCTEDefs diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala index aa13e6a67c510..838827a0e2e1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala @@ -122,7 +122,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { private def pushdownPredicatesAndAttributes( plan: LogicalPlan, cteMap: CTEMap): LogicalPlan = plan.transformWithSubqueries { - case cteDef @ CTERelationDef(child, id, originalPlanWithPredicates, _) => + case cteDef @ CTERelationDef(child, id, originalPlanWithPredicates, _, _, _) => val (_, _, newPreds, newAttrSet) = cteMap(id) val originalPlan = originalPlanWithPredicates.map(_._1).getOrElse(child) val preds = originalPlanWithPredicates.map(_._2).getOrElse(Seq.empty) @@ -141,7 +141,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { cteDef } - case cteRef @ CTERelationRef(cteId, _, output, _, _) => + case cteRef @ CTERelationRef(cteId, _, output, _, _, _) => val (cteDef, _, _, newAttrSet) = cteMap(cteId) if (needsPruning(cteDef.child, newAttrSet)) { val indices = newAttrSet.toSeq.map(cteDef.output.indexOf) @@ -170,7 +170,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { object CleanUpTempCTEInfo extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(CTE)) { - case cteDef @ CTERelationDef(_, _, Some(_), _) => + case cteDef @ CTERelationDef(_, _, Some(_), _, _, _) => cteDef.copy(originalPlanWithPredicates = None) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index f2a46b57b478f..540a5580622f2 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 @@ -801,10 +801,12 @@ object View { * @param child The final query of this CTE. * @param cteRelations A sequence of pair (alias, the CTE definition) that this CTE defined * Each CTE can see the base tables and the previously defined CTEs only. + * @param allowRecursion A boolean flag if recursion is allowed. */ case class UnresolvedWith( child: LogicalPlan, - cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { + cteRelations: Seq[(String, SubqueryAlias)], + allowRecursion: Boolean = false) extends UnaryNode { final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_WITH) override def output: Seq[Attribute] = child.output @@ -830,12 +832,19 @@ case class UnresolvedWith( * pushdown to help ensure rule idempotency. * @param underSubquery If true, it means we don't need to add a shuffle for this CTE relation as * subquery reuse will be applied to reuse CTE relation output. + * @param recursive If true, then this CTE Definition is recursive - it contains a self-reference. + * @param recursionAnchor A helper plan node that temporary stores the anchor term of recursive + * definitions. In the beginning of recursive resolution the `ResolveWithCTE` + * rule updates this parameter and once it is resolved the same rule resolves + * the recursive [[CTERelationRef]] references and removes this parameter. */ case class CTERelationDef( child: LogicalPlan, id: Long = CTERelationDef.newId, originalPlanWithPredicates: Option[(LogicalPlan, Seq[Expression])] = None, - underSubquery: Boolean = false) extends UnaryNode { + underSubquery: Boolean = false, + recursive: Boolean = false, + recursionAnchor: Option[LogicalPlan] = None) extends UnaryNode { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) @@ -859,13 +868,15 @@ object CTERelationDef { * de-duplication. * @param statsOpt The optional statistics inferred from the corresponding CTE * definition. + * @param recursive If this is a recursive reference. */ case class CTERelationRef( cteId: Long, _resolved: Boolean, override val output: Seq[Attribute], override val isStreaming: Boolean, - statsOpt: Option[Statistics] = None) extends LeafNode with MultiInstanceRelation { + statsOpt: Option[Statistics] = None, + recursive: Boolean = false) extends LeafNode with MultiInstanceRelation { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out index 0b539267e720f..926e3cd003046 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out @@ -4,13 +4,13 @@ CREATE TABLE cte_tbl USING csv AS WITH s AS (SELECT 42 AS col) SELECT * FROM s -- !query analysis CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`cte_tbl`, ErrorIfExists, [col] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -26,13 +26,13 @@ CREATE TEMPORARY VIEW cte_view AS WITH s AS (SELECT 42 AS col) SELECT * FROM s -- !query analysis CreateViewCommand `cte_view`, WITH s AS (SELECT 42 AS col) SELECT * FROM s, false, false, LocalTempView, UNSUPPORTED, true +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -43,13 +43,13 @@ Project [col#x] +- View (`cte_view`, [col#x]) +- Project [cast(col#x as int) AS col#x] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -58,13 +58,13 @@ INSERT INTO cte_tbl SELECT * FROM S -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/cte_tbl, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/cte_tbl], Append, `spark_catalog`.`default`.`cte_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/cte_tbl), [col] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [43 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias S - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -80,13 +80,13 @@ INSERT INTO cte_tbl WITH s AS (SELECT 44 AS col) SELECT * FROM s -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/cte_tbl, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/cte_tbl], Append, `spark_catalog`.`default`.`cte_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/cte_tbl), [col] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [44 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out index 3a9fc5ea1297f..0365c4dcd8e09 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out @@ -7,18 +7,18 @@ WITH t as ( SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -30,14 +30,14 @@ SELECT max(c) FROM ( Aggregate [max(c#x) AS max(c)#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -48,13 +48,13 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [1 AS 1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- OneRowRelation @@ -66,7 +66,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias unreferenced : : +- Project [outer(id#xL)] : : +- OneRowRelation @@ -83,7 +83,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias unreferenced : : +- Project [1 AS 1#x] : : +- OneRowRelation @@ -128,22 +128,22 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [2#x], false +: +- CTERelationRef xxxx, true, [2#x], false, false +- Project [2#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [2#x], false + +- CTERelationRef xxxx, true, [2#x], false, false -- !query @@ -160,29 +160,29 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- Aggregate [max(c#x) AS max(c)#x] : : +- SubqueryAlias __auto_generated_subquery_name : : +- WithCTE -: : :- CTERelationDef xxxx, false +: : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [2#x AS c#x] : : : +- Project [2 AS 2#x] : : : +- OneRowRelation : : +- Project [c#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c#x], false +: : +- CTERelationRef xxxx, true, [c#x], false, false : +- OneRowRelation +- Project [scalarsubquery()#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [scalarsubquery()#x], false + +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false -- !query @@ -199,31 +199,31 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [3 AS 3#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [3#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [3#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [3#x], false +: +- CTERelationRef xxxx, true, [3#x], false, false +- Project [3#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [3#x], false + +- CTERelationRef xxxx, true, [3#x], false, false -- !query @@ -234,12 +234,12 @@ SELECT max(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -248,7 +248,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -261,12 +261,12 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -277,7 +277,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -291,17 +291,17 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [3#x AS c#x] : +- Project [3 AS 3#x] @@ -312,7 +312,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -323,19 +323,19 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [2#x] : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [2#x], false + : +- CTERelationRef xxxx, true, [2#x], false, false +- OneRowRelation @@ -349,20 +349,20 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- WithCTE - : : :- CTERelationDef xxxx, false + : : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [2 AS 2#x] : : : +- OneRowRelation : : +- Project [2#x] : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [2#x], false + : : +- CTERelationRef xxxx, true, [2#x], false, false : +- OneRowRelation +- OneRowRelation @@ -378,25 +378,25 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- WithCTE - : : :- CTERelationDef xxxx, false + : : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [3 AS 3#x] : : : +- OneRowRelation : : +- Project [3#x] : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [3#x], false + : : +- CTERelationRef xxxx, true, [3#x], false, false : +- OneRowRelation +- OneRowRelation @@ -410,7 +410,7 @@ WHERE c IN ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] @@ -418,16 +418,16 @@ WithCTE +- Project [c#x] +- Filter c#x IN (list#x []) : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [2#x AS c#x] : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [c#x] : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [c#x], false + : +- CTERelationRef xxxx, true, [c#x], false, false +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -440,22 +440,22 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -468,22 +468,22 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias abc : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias aBc : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x] : +- SubqueryAlias aBC -: +- CTERelationRef xxxx, true, [2#x], false +: +- CTERelationRef xxxx, true, [2#x], false, false +- Project [2#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [2#x], false + +- CTERelationRef xxxx, true, [2#x], false, false -- !query @@ -494,19 +494,19 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias abc : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias aBc : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [2#x] : +- SubqueryAlias aBC - : +- CTERelationRef xxxx, true, [2#x], false + : +- CTERelationRef xxxx, true, [2#x], false, false +- OneRowRelation @@ -522,23 +522,23 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -553,20 +553,20 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -586,27 +586,27 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias __auto_generated_subquery_name : +- Project [1#x] : +- SubqueryAlias cte_inner_inner -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out index e8640c3cbb6bd..53dcd46361dc1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out @@ -7,18 +7,18 @@ WITH t as ( SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -30,14 +30,14 @@ SELECT max(c) FROM ( Aggregate [max(c#x) AS max(c)#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -48,13 +48,13 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [1 AS 1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- OneRowRelation @@ -66,7 +66,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias unreferenced : : +- Project [outer(id#xL)] : : +- OneRowRelation @@ -83,7 +83,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias unreferenced : : +- Project [1 AS 1#x] : : +- OneRowRelation @@ -153,29 +153,29 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- Aggregate [max(c#x) AS max(c)#x] : : +- SubqueryAlias __auto_generated_subquery_name : : +- WithCTE -: : :- CTERelationDef xxxx, false +: : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [2#x AS c#x] : : : +- Project [2 AS 2#x] : : : +- OneRowRelation : : +- Project [c#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c#x], false +: : +- CTERelationRef xxxx, true, [c#x], false, false : +- OneRowRelation +- Project [scalarsubquery()#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [scalarsubquery()#x], false + +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false -- !query @@ -211,12 +211,12 @@ SELECT max(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -225,7 +225,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -238,12 +238,12 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -254,7 +254,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -268,17 +268,17 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [3#x AS c#x] : +- Project [3 AS 3#x] @@ -289,7 +289,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -384,22 +384,22 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -454,23 +454,23 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -485,20 +485,20 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -518,27 +518,27 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias __auto_generated_subquery_name : +- Project [1#x] : +- SubqueryAlias cte_inner_inner -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out index 9b86c4df62c4d..9a221f6a699cd 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out @@ -62,7 +62,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException WITH t AS (SELECT 1 FROM t) SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- SubqueryAlias t @@ -73,7 +73,7 @@ WithCTE : +- LocalRelation [id#x] +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -100,7 +100,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [id#x] : +- SubqueryAlias t2 @@ -109,17 +109,17 @@ WithCTE : +- Project [id#x] : +- SubqueryAlias t : +- LocalRelation [id#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [id#x], false +: +- CTERelationRef xxxx, true, [id#x], false, false +- Project [id#x, 2#x] +- Join Cross :- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [id#x], false + : +- CTERelationRef xxxx, true, [id#x], false, false +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [2#x], false + +- CTERelationRef xxxx, true, [2#x], false, false -- !query @@ -134,7 +134,7 @@ FROM CTE1 t1 CROSS JOIN CTE1 t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias CTE1 : +- Project [id#x AS id#x] : +- Join Cross @@ -157,10 +157,10 @@ WithCTE +- Join Cross :- SubqueryAlias t1 : +- SubqueryAlias CTE1 - : +- CTERelationRef xxxx, true, [id#x], false + : +- CTERelationRef xxxx, true, [id#x], false, false +- SubqueryAlias t2 +- SubqueryAlias CTE1 - +- CTERelationRef xxxx, true, [id#x], false + +- CTERelationRef xxxx, true, [id#x], false, false -- !query @@ -168,7 +168,7 @@ WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE x = 1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS x#x] : +- Project [1 AS 1#x] @@ -176,7 +176,7 @@ WithCTE +- Project [x#x] +- Filter (x#x = 1) +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -184,7 +184,7 @@ WITH t(x, y) AS (SELECT 1, 2) SELECT * FROM t WHERE x = 1 AND y = 2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS x#x, 2#x AS y#x] : +- Project [1 AS 1#x, 2 AS 2#x] @@ -192,7 +192,7 @@ WithCTE +- Project [x#x, y#x] +- Filter ((x#x = 1) AND (y#x = 2)) +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x, y#x], false + +- CTERelationRef xxxx, true, [x#x, y#x], false, false -- !query @@ -200,14 +200,14 @@ WITH t(x, x) AS (SELECT 1, 2) SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS x#x, 2#x AS x#x] : +- Project [1 AS 1#x, 2 AS 2#x] : +- OneRowRelation +- Project [x#x, x#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x, x#x], false + +- CTERelationRef xxxx, true, [x#x, x#x], false, false -- !query @@ -300,56 +300,56 @@ WITH w1(c1) AS SELECT * FROM w1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w8 : +- Project [1#x AS c8#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w7 : +- Project [c8#x AS c7#x] : +- Project [c8#x] : +- SubqueryAlias w8 -: +- CTERelationRef xxxx, true, [c8#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c8#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w6 : +- Project [c7#x AS c6#x] : +- Project [c7#x] : +- SubqueryAlias w7 -: +- CTERelationRef xxxx, true, [c7#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c7#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w5 : +- Project [c6#x AS c5#x] : +- Project [c6#x] : +- SubqueryAlias w6 -: +- CTERelationRef xxxx, true, [c6#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c6#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w4 : +- Project [c5#x AS c4#x] : +- Project [c5#x] : +- SubqueryAlias w5 -: +- CTERelationRef xxxx, true, [c5#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c5#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w3 : +- Project [c4#x AS c3#x] : +- Project [c4#x] : +- SubqueryAlias w4 -: +- CTERelationRef xxxx, true, [c4#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c4#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w2 : +- Project [c3#x AS c2#x] : +- Project [c3#x] : +- SubqueryAlias w3 -: +- CTERelationRef xxxx, true, [c3#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c3#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w1 : +- Project [c2#x AS c1#x] : +- Project [c2#x] : +- SubqueryAlias w2 -: +- CTERelationRef xxxx, true, [c2#x], false +: +- CTERelationRef xxxx, true, [c2#x], false, false +- Project [c1#x] +- SubqueryAlias w1 - +- CTERelationRef xxxx, true, [c1#x], false + +- CTERelationRef xxxx, true, [c1#x], false, false -- !query @@ -379,14 +379,14 @@ WITH same_name AS (SELECT 42) SELECT * FROM same_name, (SELECT 10) AS same_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias same_name : +- Project [42 AS 42#x] : +- OneRowRelation +- Project [42#x, 10#x] +- Join Inner :- SubqueryAlias same_name - : +- CTERelationRef xxxx, true, [42#x], false + : +- CTERelationRef xxxx, true, [42#x], false, false +- SubqueryAlias same_name +- Project [10 AS 10#x] +- OneRowRelation @@ -419,13 +419,13 @@ WITH q AS (SELECT 'foo' AS x) SELECT x, typeof(x) FROM q -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q : +- Project [foo AS x#x] : +- OneRowRelation +- Project [x#x, typeof(x#x) AS typeof(x)#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -479,13 +479,13 @@ SELECT * FROM Project [y#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias q : +- Project [1 AS x#x] : +- OneRowRelation +- Project [(x#x + 1) AS y#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -493,13 +493,13 @@ select (with q as (select 1 x) select * from q) -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [1 AS x#x] : : +- OneRowRelation : +- Project [x#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- OneRowRelation @@ -508,13 +508,13 @@ select 1 in (with q as (select 1) select * from q) -- !query analysis Project [1 IN (list#x []) AS (1 IN (listquery()))#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [1 AS 1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- OneRowRelation @@ -549,11 +549,11 @@ from T1 z -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T1 : +- Project [1 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T1 : +- Project [2 AS b#x] : +- OneRowRelation @@ -562,14 +562,14 @@ WithCTE :- Join Inner : :- SubqueryAlias x : : +- SubqueryAlias T1 - : : +- CTERelationRef xxxx, true, [a#x], false + : : +- CTERelationRef xxxx, true, [a#x], false, false : +- SubqueryAlias y : +- Project [b#x] : +- SubqueryAlias T1 - : +- CTERelationRef xxxx, true, [b#x], false + : +- CTERelationRef xxxx, true, [b#x], false, false +- SubqueryAlias z +- SubqueryAlias T1 - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query @@ -580,15 +580,15 @@ from (WITH TtTt as (select 3 c) select * from ttTT, `tttT_2`) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias TTtt : +- Project [1 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias tTTt_2 : +- Project [2 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias TtTt : +- Project [3 AS c#x] : +- OneRowRelation @@ -597,9 +597,9 @@ WithCTE +- Project [c#x, a#x] +- Join Inner :- SubqueryAlias ttTT - : +- CTERelationRef xxxx, true, [c#x], false + : +- CTERelationRef xxxx, true, [c#x], false, false +- SubqueryAlias tttT_2 - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query @@ -609,13 +609,13 @@ from (select 1 x, 2 y) T -- !query analysis Project [scalar-subquery#x [x#x] AS scalarsubquery(x)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [outer(x#x)] : : +- OneRowRelation : +- Project [x#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- SubqueryAlias T +- Project [1 AS x#x, 2 AS y#x] +- OneRowRelation @@ -628,13 +628,13 @@ from (select 1 x, 2 y) T -- !query analysis Project [scalar-subquery#x [x#x && y#x] AS scalarsubquery(x, y)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [3 AS z#x] : : +- OneRowRelation : +- Project [((outer(x#x) + outer(y#x)) + z#x) AS ((outer(T.x) + outer(T.y)) + z)#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [z#x], false +: +- CTERelationRef xxxx, true, [z#x], false, false +- SubqueryAlias T +- Project [1 AS x#x, 2 AS y#x] +- OneRowRelation @@ -646,20 +646,20 @@ select * from (with q2 as (select * from q1) select * from q2) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [1 AS x#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q2 : +- Project [x#x] : +- SubqueryAlias q1 -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- Project [x#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [x#x] +- SubqueryAlias q2 - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -668,20 +668,20 @@ select * from (with q1 as (select x+1 from q1) select * from q1) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [1 AS x#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [(x#x + 1) AS (x + 1)#x] : +- SubqueryAlias q1 -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- Project [(x + 1)#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [(x + 1)#x] +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [(x + 1)#x], false + +- CTERelationRef xxxx, true, [(x + 1)#x], false, false -- !query @@ -709,7 +709,7 @@ with cte1 as (Select id as j from t) select * from cte1 where j = (select max(j) from cte1 as cte2) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [id#x AS j#x] : +- SubqueryAlias t @@ -723,9 +723,9 @@ WithCTE : +- Aggregate [max(j#x) AS max(j)#x] : +- SubqueryAlias cte2 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [j#x], false + : +- CTERelationRef xxxx, true, [j#x], false, false +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [j#x], false + +- CTERelationRef xxxx, true, [j#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out index 22dfeac5fd0b6..35713167bc38d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out @@ -411,14 +411,14 @@ CREATE TEMPORARY VIEW "myview"("c1") AS -- !query analysis CreateViewCommand `myview`, [(c1,None)], WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v", false, false, LocalTempView, UNSUPPORTED, true +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias v : +- Project [1#x AS a#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [a#x] +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query @@ -431,14 +431,14 @@ Project [a1#x AS a2#x] +- View (`myview`, [c1#x]) +- Project [cast(a#x as int) AS c1#x] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias v : +- Project [1#x AS a#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [a#x] +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index 20e6ca1e6a2ec..9384f17fa74c1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -1012,17 +1012,17 @@ WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias S : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] +- Aggregate [max(c1#x) AS max(c1)#x] +- SubqueryAlias T - +- CTERelationRef xxxx, true, [c1#x, c2#x], false + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false -- !query @@ -1031,17 +1031,17 @@ WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T') -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias S : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] +- Aggregate [max(c1#x) AS max(c1)#x] +- SubqueryAlias T - +- CTERelationRef xxxx, true, [c1#x, c2#x], false + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false -- !query @@ -1049,13 +1049,13 @@ WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC') -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias ABC : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] +- Aggregate [max(c1#x) AS max(c1)#x] +- SubqueryAlias ABC - +- CTERelationRef xxxx, true, [c1#x, c2#x], false + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index 5bf893605423c..c93109e428362 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -1358,14 +1358,14 @@ WITH cte1 AS ( SELECT * FROM cte2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [c1#x] : +- SubqueryAlias spark_catalog.default.t1 : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x]) : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : +- LocalRelation [col1#x, col2#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte2 : +- Project [c1#x, c2#x] : +- LateralJoin lateral-subquery#x [c1#x], Inner @@ -1377,10 +1377,10 @@ WithCTE : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : : +- LocalRelation [col1#x, col2#x] : +- SubqueryAlias cte1 -: +- CTERelationRef xxxx, true, [c1#x], false +: +- CTERelationRef xxxx, true, [c1#x], false, false +- Project [c1#x, c2#x] +- SubqueryAlias cte2 - +- CTERelationRef xxxx, true, [c1#x, c2#x], false + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out index 6b2c60f25bae3..7f1ca3dda902d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out @@ -36,7 +36,7 @@ WITH tmp AS ( SELECT id FROM range(3) WHERE id > (SELECT max(id) FROM tmp) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias tmp : +- Intersect false : :- Project [id#xL] @@ -47,7 +47,7 @@ WithCTE +- Filter (id#xL > scalar-subquery#x []) : +- Aggregate [max(id#xL) AS max(id)#xL] : +- SubqueryAlias tmp - : +- CTERelationRef xxxx, true, [id#xL], false + : +- CTERelationRef xxxx, true, [id#xL], false, false +- Range (0, 3, step=1) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out index 9f2dd9bcb1783..441b3627bc928 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out @@ -89,7 +89,7 @@ FROM cte WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [id#xL AS x#xL] : +- Project [id#xL] @@ -99,7 +99,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query @@ -111,7 +111,7 @@ FROM cte WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [id#xL AS x#xL] : +- Project [id#xL] @@ -121,7 +121,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query @@ -134,7 +134,7 @@ FROM cte WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [1#xL AS x#xL] : +- Union false, false @@ -154,7 +154,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query @@ -167,7 +167,7 @@ FROM cte WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [1#xL AS x#xL] : +- Union false, false @@ -187,7 +187,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out index f29b12d5af171..b92e70e91a4ba 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out @@ -4,7 +4,7 @@ WITH q1(x,y) AS (SELECT 1,2) SELECT * FROM q1, q1 AS q2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [1#x AS x#x, 2#x AS y#x] : +- Project [1 AS 1#x, 2 AS 2#x] @@ -12,10 +12,10 @@ WithCTE +- Project [x#x, y#x, x#x, y#x] +- Join Inner :- SubqueryAlias q1 - : +- CTERelationRef xxxx, true, [x#x, y#x], false + : +- CTERelationRef xxxx, true, [x#x, y#x], false, false +- SubqueryAlias q2 +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [x#x, y#x], false + +- CTERelationRef xxxx, true, [x#x, y#x], false, false -- !query @@ -185,7 +185,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [42#x AS foo#x] : +- Project [42 AS 42#x] @@ -194,7 +194,7 @@ WithCTE +- SubqueryAlias q +- Project [foo#x] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [foo#x], false + +- CTERelationRef xxxx, true, [foo#x], false, false -- !query @@ -207,11 +207,11 @@ WITH outermost(x) AS ( SELECT * FROM outermost ORDER BY 1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias innermost : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias outermost : +- Project [1#x AS x#x] : +- Distinct @@ -222,13 +222,13 @@ WithCTE : +- Union false, false : :- Project [2#x] : : +- SubqueryAlias innermost -: : +- CTERelationRef xxxx, true, [2#x], false +: : +- CTERelationRef xxxx, true, [2#x], false, false : +- Project [3 AS 3#x] : +- OneRowRelation +- Sort [x#x ASC NULLS FIRST], true +- Project [x#x] +- SubqueryAlias outermost - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -412,13 +412,13 @@ org.apache.spark.sql.catalyst.parser.ParseException with ordinality as (select 1 as x) select * from ordinality -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias ordinality : +- Project [1 AS x#x] : +- OneRowRelation +- Project [x#x] +- SubqueryAlias ordinality - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -453,13 +453,13 @@ with test as (select 42) insert into test select * from test InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test], Append, `spark_catalog`.`default`.`test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test), [i] +- Project [cast(42#x as int) AS i#x] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias test : +- Project [42 AS 42#x] : +- OneRowRelation +- Project [42#x] +- SubqueryAlias test - +- CTERelationRef xxxx, true, [42#x], false + +- CTERelationRef xxxx, true, [42#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out index 02e7c39ae83fd..da4542fe9bb1c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out @@ -2044,13 +2044,13 @@ Project [1 AS 1#x] WITH v1 AS (SELECT var1 AS c1) SELECT c1 AS `1` FROM v1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias v1 : +- Project [variablereference(system.session.var1=1) AS c1#x] : +- OneRowRelation +- Project [c1#x AS 1#x] +- SubqueryAlias v1 - +- CTERelationRef xxxx, true, [c1#x], false + +- CTERelationRef xxxx, true, [c1#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out index 7c3678c66c117..f210bc49030fc 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out @@ -101,7 +101,7 @@ WHERE a.bonus_amt > 30 WHERE a.emp_name = b.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias bonus_cte : +- Project [emp_name#x, bonus_amt#x] : +- Filter exists#x [emp_name#x] @@ -133,7 +133,7 @@ WithCTE : +- Filter (outer(emp_name#x) = emp_name#x) : +- SubqueryAlias b : +- SubqueryAlias bonus_cte - : +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], false + : +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], false, false +- SubqueryAlias a +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) @@ -162,7 +162,7 @@ WHERE EXISTS (SELECT * WHERE bonus.emp_name = a.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias emp_cte : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- Filter ((id#x >= 100) AND (id#x <= 300)) @@ -172,7 +172,7 @@ WithCTE : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- SubqueryAlias EMP : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias dept_cte : +- Project [dept_id#x, dept_name#x, state#x] : +- Filter (dept_id#x = 10) @@ -189,10 +189,10 @@ WithCTE : +- Join Inner, (dept_id#x = dept_id#x) : :- SubqueryAlias a : : +- SubqueryAlias emp_cte - : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false : +- SubqueryAlias b : +- SubqueryAlias dept_cte - : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] @@ -225,7 +225,7 @@ WHERE e.dept_id = d.dept_id WHERE e.emp_name = a.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias emp_cte : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- Filter ((id#x >= 100) AND (id#x <= 300)) @@ -235,7 +235,7 @@ WithCTE : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- SubqueryAlias EMP : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias dept_cte : +- Project [dept_id#x, dept_name#x, state#x] : +- Filter (dept_id#x = 10) @@ -253,10 +253,10 @@ WithCTE : +- Join LeftOuter, (dept_id#x = dept_id#x) : :- SubqueryAlias a : : +- SubqueryAlias emp_cte - : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false : +- SubqueryAlias b : +- SubqueryAlias dept_cte - : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false +- Join Inner :- Join Inner : :- SubqueryAlias b @@ -268,7 +268,7 @@ WithCTE : : +- LocalRelation [emp_name#x, bonus_amt#x] : +- SubqueryAlias e : +- SubqueryAlias emp_cte - : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false + : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false +- SubqueryAlias d +- SubqueryAlias dept +- View (`DEPT`, [dept_id#x, dept_name#x, state#x]) @@ -299,7 +299,7 @@ WHERE EXISTS (SELECT dept_id, GROUP BY emp_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias empdept : +- Project [id#x, salary#x, emp_name#x, dept_id#x] : +- Filter id#x IN (100,200) @@ -322,7 +322,7 @@ WithCTE : +- Filter (count(1)#xL > cast(1 as bigint)) : +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] : +- SubqueryAlias empdept - : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] @@ -352,7 +352,7 @@ WHERE NOT EXISTS (SELECT dept_id, GROUP BY emp_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias empdept : +- Project [id#x, salary#x, emp_name#x, dept_id#x] : +- Filter id#x IN (100,200) @@ -375,7 +375,7 @@ WithCTE : +- Filter (count(1)#xL < cast(1 as bigint)) : +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] : +- SubqueryAlias empdept - : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out index 39748a324e527..85e28de736a0c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out @@ -307,7 +307,7 @@ FROM ( on cte1.t1b = cte2.t1b) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter named_struct(t1b, t1b#x, t1d, t1d#xL) IN (list#x [t1c#x]) @@ -330,7 +330,7 @@ WithCTE +- Project [t1a#x, t1b#x, t1a#x, t1b#x] +- Join Inner, (t1b#x = t1b#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out index 0074991b4ea6a..03a4c3ffd4fcb 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out @@ -123,7 +123,7 @@ WHERE t1b IN (SELECT cte1.t1b WHERE cte1.t1b > 0) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1a#x = val1a) @@ -138,7 +138,7 @@ WithCTE : +- Project [t1b#x] : +- Filter (cast(t1b#x as int) > 0) : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -177,7 +177,7 @@ GROUP BY t1a, t1b, t1c HAVING t1c IS NOT NULL -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- SubqueryAlias t1 @@ -197,21 +197,21 @@ WithCTE : : : :- Project [t1b#x] : : : : +- Filter (cast(t1b#x as int) > 0) : : : : +- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : : +- Project [t1b#x] : : : +- Filter (cast(t1b#x as int) > 5) : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : +- Intersect false : : :- Project [t1b#x] : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : +- Project [t1b#x] : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : +- Project [t1b#x] : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -250,7 +250,7 @@ WHERE t1c IN ON cte1.t1d > cte6.t1d) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] : +- SubqueryAlias t1 @@ -268,22 +268,22 @@ WithCTE : : : :- Join FullOuter, (t1c#x = t1c#x) : : : : :- Join Inner, (t1b#x > t1b#x) : : : : : :- SubqueryAlias cte1 - : : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : : : : +- SubqueryAlias cte2 : : : : : +- SubqueryAlias cte1 - : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : : : +- SubqueryAlias cte3 : : : : +- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : : +- SubqueryAlias cte4 : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : +- SubqueryAlias cte5 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : +- SubqueryAlias cte6 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -316,7 +316,7 @@ FROM (SELECT * ON cte1.t1b = cte4.t1b) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1b#x IN (list#x []) AND (t1a#x = val1b)) @@ -354,16 +354,16 @@ WithCTE :- Join FullOuter, (t1a#x = t1a#x) : :- Join Inner, ((cast(t1b#x as int) > 5) AND (t1a#x = t1a#x)) : : :- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : +- SubqueryAlias cte2 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : +- SubqueryAlias cte3 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte4 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false -- !query @@ -393,7 +393,7 @@ WHERE t1b IN GROUP BY t1b -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1h#x] : +- Filter t1a#x IN (list#x [t1b#x]) @@ -424,10 +424,10 @@ WithCTE +- Project [t1a#x, t1b#x] +- Join Inner, (t1h#x >= t1h#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false -- !query @@ -453,7 +453,7 @@ FROM ( ) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x] : +- Filter (t1b#x IN (list#x [t1c#x]) AND (t1a#x = val1b)) @@ -485,16 +485,16 @@ WithCTE :- Join RightOuter, (t1b#x = t1b#x) : :- Join Inner, (t1a#x = t1a#x) : : :- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false : : +- SubqueryAlias cte2 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false : +- SubqueryAlias cte3 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false +- SubqueryAlias cte4 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false -- !query @@ -515,7 +515,7 @@ FROM (SELECT cte1.t1a, GROUP BY s.t1b -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter t1b#x IN (list#x [t1c#x]) @@ -538,10 +538,10 @@ WithCTE +- Project [t1a#x, t1b#x] +- Join RightOuter, (t1a#x = t1a#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false -- !query @@ -569,7 +569,7 @@ WHERE s.t1b IN ON t1.t1a = cte1.t1a) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter t1b#x IN (list#x [t1c#x]) @@ -599,15 +599,15 @@ WithCTE : : +- SubqueryAlias t1 : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias s +- Project [t1b#x] +- Join LeftOuter, (t1b#x = t1b#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false -- !query @@ -627,7 +627,7 @@ WHERE t1b NOT IN (SELECT cte1.t1b t1c > 10 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1a#x = val1d) @@ -642,7 +642,7 @@ WithCTE : +- Project [t1b#x] : +- Filter (cast(t1b#x as int) < 0) : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -683,7 +683,7 @@ WHERE t1b NOT IN ORDER BY t1c DESC -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] : +- Filter NOT t1d#xL IN (list#x []) @@ -722,16 +722,16 @@ WithCTE : :- Join RightOuter, (t1b#x = t1b#x) : : :- Join Inner, (t1a#x = t1a#x) : : : :- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false : : : +- SubqueryAlias cte2 : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false : : +- SubqueryAlias cte3 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false : +- SubqueryAlias cte4 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out index 72e230f9bb881..3b1ffa94c17f2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -617,13 +617,13 @@ SELECT c1, (WITH t AS (SELECT 1 AS a) SELECT a + c1 FROM t) FROM t1 -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [1 AS a#x] : : +- OneRowRelation : +- Project [(a#x + outer(c1#x)) AS (a + outer(t1.c1))#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [a#x], false +: +- CTERelationRef xxxx, true, [a#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -636,7 +636,7 @@ SELECT c1, (WITH t AS (SELECT * FROM t2 WHERE c1 = t1.c1) SELECT SUM(c2) FROM t) -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- Filter (c1#x = outer(c1#x)) @@ -647,7 +647,7 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : : +- LocalRelation [c1#x, c2#x] : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -664,7 +664,7 @@ SELECT c1, ( -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t3 : : +- Project [(c1#x + 1) AS c1#x, (c2#x + 1) AS c2#x] : : +- SubqueryAlias t2 @@ -672,15 +672,15 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] : : +- SubqueryAlias t2 : : +- LocalRelation [c1#x, c2#x] -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t4 : : +- Project [c1#x, c2#x] : : +- Filter (outer(c1#x) = c1#x) : : +- SubqueryAlias t3 -: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : +- SubqueryAlias t4 -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -697,7 +697,7 @@ SELECT c1, ( -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- SubqueryAlias t2 @@ -713,10 +713,10 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- Union false, false : :- Project [c1#x, c2#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false : +- Project [c2#x, c1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -732,7 +732,7 @@ SELECT * FROM t1 WHERE c1 > ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias v : +- Project [c1#x, c2#x] : +- SubqueryAlias t2 @@ -743,7 +743,7 @@ WithCTE +- Project [c1#x, c2#x] +- Filter (cast(c1#x as bigint) > scalar-subquery#x [c1#x]) : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- SubqueryAlias t2 @@ -756,9 +756,9 @@ WithCTE : : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : : +- Filter (c1#x = outer(c1#x)) : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [c1#x, c2#x], false + : : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false : +- SubqueryAlias v - : +- CTERelationRef xxxx, true, [c1#x, c2#x], false + : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -771,7 +771,7 @@ WITH t AS (SELECT 1 AS a) SELECT c1, (SELECT a FROM t WHERE a = c1) FROM t1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS a#x] : +- OneRowRelation @@ -779,7 +779,7 @@ WithCTE : +- Project [a#x] : +- Filter (a#x = outer(c1#x)) : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [a#x], false + : +- CTERelationRef xxxx, true, [a#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -1019,7 +1019,7 @@ SELECT (SELECT sum(1) FROM T WHERE a = col OR upper(col)= 'Y') FROM (SELECT null as col) as foo -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T : +- Project [1 AS a#x] : +- OneRowRelation @@ -1027,7 +1027,7 @@ WithCTE : +- Aggregate [sum(1) AS sum(1)#xL] : +- Filter ((a#x = cast(outer(col#x) as int)) OR (upper(cast(outer(col#x) as string)) = Y)) : +- SubqueryAlias T - : +- CTERelationRef xxxx, true, [a#x], false + : +- CTERelationRef xxxx, true, [a#x], false, false +- SubqueryAlias foo +- Project [null AS col#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out index f0d3b8b999a29..04e019fdbaa41 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out @@ -874,7 +874,7 @@ WITH temp AS ( SELECT t1.b FROM temp t1 JOIN temp t2 ON t1.b = t2.b -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias temp : +- ScriptTransformation cat, [b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) : +- Project [a#x] @@ -888,10 +888,10 @@ WithCTE +- Join Inner, (b#x = b#x) :- SubqueryAlias t1 : +- SubqueryAlias temp - : +- CTERelationRef xxxx, true, [b#x], false + : +- CTERelationRef xxxx, true, [b#x], false, false +- SubqueryAlias t2 +- SubqueryAlias temp - +- CTERelationRef xxxx, true, [b#x], false + +- CTERelationRef xxxx, true, [b#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out index 5a74c4be107e3..89f988fe2b616 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out @@ -817,12 +817,12 @@ FROM t1 FULL OUTER JOIN t2 USING (key) WHERE t1.key NOT LIKE 'bb.%' -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [key#x] : +- SubqueryAlias t : +- LocalRelation [key#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [key#x] : +- SubqueryAlias t @@ -833,6 +833,6 @@ WithCTE +- Project [coalesce(key#x, key#x) AS key#x, key#x, key#x, key#x] +- Join FullOuter, (key#x = key#x) :- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [key#x], false + : +- CTERelationRef xxxx, true, [key#x], false, false +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [key#x], false + +- CTERelationRef xxxx, true, [key#x], false, false