diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/WithCTENormalized.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/WithCTENormalized.scala new file mode 100644 index 000000000000..9e7675b64962 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/WithCTENormalized.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.normalizer + +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{CacheTableAsSelect, CTERelationRef, LogicalPlan, WithCTE} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.{CTE, PLAN_EXPRESSION} + +object WithCTENormalized extends Rule[LogicalPlan]{ + override def apply(plan: LogicalPlan): LogicalPlan = { + val curId = new java.util.concurrent.atomic.AtomicLong() + plan transformDown { + + case ctas @ CacheTableAsSelect(_, plan, _, _, _, _, _) => + ctas.copy(plan = apply(plan)) + + case withCTE @ WithCTE(plan, cteDefs) => + val defIdToNewId = withCTE.cteDefs.map(_.id).map((_, curId.getAndIncrement())).toMap + val normalizedPlan = canonicalizeCTE(plan, defIdToNewId) + val newCteDefs = cteDefs.map { cteDef => + val normalizedCteDef = canonicalizeCTE(cteDef.child, defIdToNewId) + cteDef.copy(child = normalizedCteDef, id = defIdToNewId(cteDef.id)) + } + withCTE.copy(plan = normalizedPlan, cteDefs = newCteDefs) + } + } + + def canonicalizeCTE(plan: LogicalPlan, defIdToNewId: Map[Long, Long]): LogicalPlan = { + plan.transformDownWithPruning( + _.containsAnyPattern(CTE, PLAN_EXPRESSION)) { + // For nested WithCTE, if defIndex didn't contain the cteId, + // means it's not current WithCTE's ref. + case ref: CTERelationRef if defIdToNewId.contains(ref.cteId) => + ref.copy(cteId = defIdToNewId(ref.cteId)) + case other => + other.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { + case e: SubqueryExpression => e.withNewPlan(canonicalizeCTE(e.plan, defIdToNewId)) + } + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 72274ee9bf17..fab64d771093 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -1742,7 +1742,8 @@ case class CacheTableAsSelect( isLazy: Boolean, options: Map[String, String], isAnalyzed: Boolean = false, - referredTempFunctions: Seq[String] = Seq.empty) extends AnalysisOnlyCommand { + referredTempFunctions: Seq[String] = Seq.empty) + extends AnalysisOnlyCommand with CTEInChildren { override protected def withNewChildrenInternal( newChildren: IndexedSeq[LogicalPlan]): CacheTableAsSelect = { assert(!isAnalyzed) @@ -1757,6 +1758,10 @@ case class CacheTableAsSelect( // Collect the referred temporary functions from AnalysisContext referredTempFunctions = ac.referredTempFunctionNames.toSeq) } + + override def withCTEDefs(cteDefs: Seq[CTERelationDef]): LogicalPlan = { + copy(plan = WithCTE(plan, cteDefs)) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 00c9a26cb5bf..f2cd4f3f3653 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTr import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension import org.apache.spark.sql.catalyst.catalog.{FunctionExpressionBuilder, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{Expression, ExtractSemiStructuredFields} +import org.apache.spark.sql.catalyst.normalizer.WithCTENormalized import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -403,6 +404,7 @@ abstract class BaseSessionStateBuilder( } protected def planNormalizationRules: Seq[Rule[LogicalPlan]] = { + WithCTENormalized +: extensions.buildPlanNormalizationRules(session) } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cache.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cache.sql.out new file mode 100644 index 000000000000..e93da8b86b89 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cache.sql.out @@ -0,0 +1,184 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (0, 0), (1, 1), (2, 2) AS t(c1, c2) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES (0, 0), (1, 1), (2, 2) AS t(c1, c2), false, false, LocalTempView, UNSUPPORTED, true + +- Project [c1#x, c2#x] + +- SubqueryAlias t + +- LocalRelation [c1#x, c2#x] + + +-- !query +CREATE TEMPORARY VIEW t2 AS +WITH v as ( + SELECT c1 + c1 c3 FROM t1 +) +SELECT SUM(c3) s FROM v +-- !query analysis +CreateViewCommand `t2`, WITH v as ( + SELECT c1 + c1 c3 FROM t1 +) +SELECT SUM(c3) s FROM v, false, false, LocalTempView, UNSUPPORTED, true + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias v + : +- Project [(c1#x + c1#x) AS c3#x] + : +- 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] + : +- Project [c1#x, c2#x] + : +- SubqueryAlias t + : +- LocalRelation [c1#x, c2#x] + +- Aggregate [sum(c3#x) AS s#xL] + +- SubqueryAlias v + +- CTERelationRef xxxx, true, [c3#x], false, false + + +-- !query +CACHE TABLE cache_table +WITH +t2 AS (SELECT 1) +SELECT * FROM t2 +-- !query analysis +CacheTableAsSelect cache_table, WITH +t2 AS (SELECT 1) +SELECT * FROM t2, false, true + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias t2 + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [1#x] + +- SubqueryAlias t2 + +- CTERelationRef xxxx, true, [1#x], false, false, 1 + + +-- !query +SELECT * FROM cache_table +-- !query analysis +Project [1#x] ++- SubqueryAlias cache_table + +- View (`cache_table`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias t2 + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [1#x] + +- SubqueryAlias t2 + +- CTERelationRef xxxx, true, [1#x], false, false, 1 + + +-- !query +EXPLAIN EXTENDED SELECT * FROM cache_table +-- !query analysis +ExplainCommand 'Project [*], ExtendedMode + + +-- !query +CACHE TABLE cache_nested_cte_table +WITH +v AS ( + SELECT c1 * c2 c3 from t1 +) +SELECT SUM(c3) FROM v +EXCEPT +SELECT s FROM t2 +-- !query analysis +CacheTableAsSelect cache_nested_cte_table, WITH +v AS ( + SELECT c1 * c2 c3 from t1 +) +SELECT SUM(c3) FROM v +EXCEPT +SELECT s FROM t2, false, true + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias v + : +- Project [(c1#x * c2#x) AS c3#x] + : +- 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] + : +- Project [c1#x, c2#x] + : +- SubqueryAlias t + : +- LocalRelation [c1#x, c2#x] + +- Except false + :- Aggregate [sum(c3#x) AS sum(c3)#xL] + : +- SubqueryAlias v + : +- CTERelationRef xxxx, true, [c3#x], false, false + +- Project [s#xL] + +- SubqueryAlias t2 + +- View (`t2`, [s#xL]) + +- Project [cast(s#xL as bigint) AS s#xL] + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias v + : +- Project [(c1#x + c1#x) AS c3#x] + : +- 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] + : +- Project [c1#x, c2#x] + : +- SubqueryAlias t + : +- LocalRelation [c1#x, c2#x] + +- Aggregate [sum(c3#x) AS s#xL] + +- SubqueryAlias v + +- CTERelationRef xxxx, true, [c3#x], false, false + + +-- !query +SELECT * FROM cache_nested_cte_table +-- !query analysis +Project [sum(c3)#xL] ++- SubqueryAlias cache_nested_cte_table + +- View (`cache_nested_cte_table`, [sum(c3)#xL]) + +- Project [cast(sum(c3)#xL as bigint) AS sum(c3)#xL] + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias v + : +- Project [(c1#x * c2#x) AS c3#x] + : +- 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] + : +- Project [c1#x, c2#x] + : +- SubqueryAlias t + : +- LocalRelation [c1#x, c2#x] + +- Except false + :- Aggregate [sum(c3#x) AS sum(c3)#xL] + : +- SubqueryAlias v + : +- CTERelationRef xxxx, true, [c3#x], false, false + +- Project [s#xL] + +- SubqueryAlias t2 + +- View (`t2`, [s#xL]) + +- Project [cast(s#xL as bigint) AS s#xL] + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias v + : +- Project [(c1#x + c1#x) AS c3#x] + : +- 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] + : +- Project [c1#x, c2#x] + : +- SubqueryAlias t + : +- LocalRelation [c1#x, c2#x] + +- Aggregate [sum(c3#x) AS s#xL] + +- SubqueryAlias v + +- CTERelationRef xxxx, true, [c3#x], false, false + + +-- !query +EXPLAIN EXTENDED SELECT * FROM cache_nested_cte_table +-- !query analysis +ExplainCommand 'Project [*], ExtendedMode + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query analysis +DropTempViewCommand t1 + + +-- !query +DROP TABLE IF EXISTS t2 +-- !query analysis +DropTempViewCommand t2 diff --git a/sql/core/src/test/resources/sql-tests/inputs/cache.sql b/sql/core/src/test/resources/sql-tests/inputs/cache.sql new file mode 100644 index 000000000000..b3588ce49e59 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cache.sql @@ -0,0 +1,33 @@ +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (0, 0), (1, 1), (2, 2) AS t(c1, c2); +CREATE TEMPORARY VIEW t2 AS +WITH v as ( + SELECT c1 + c1 c3 FROM t1 +) +SELECT SUM(c3) s FROM v; + +CACHE TABLE cache_table +WITH +t2 AS (SELECT 1) +SELECT * FROM t2; + +SELECT * FROM cache_table; + +EXPLAIN EXTENDED SELECT * FROM cache_table; + +-- Nested WithCTE +CACHE TABLE cache_nested_cte_table +WITH +v AS ( + SELECT c1 * c2 c3 from t1 +) +SELECT SUM(c3) FROM v +EXCEPT +SELECT s FROM t2; + +SELECT * FROM cache_nested_cte_table; + +EXPLAIN EXTENDED SELECT * FROM cache_nested_cte_table; + +-- Clean up +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/sql/core/src/test/resources/sql-tests/results/cache.sql.out b/sql/core/src/test/resources/sql-tests/results/cache.sql.out new file mode 100644 index 000000000000..f7c3ec6d9f03 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cache.sql.out @@ -0,0 +1,293 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (0, 0), (1, 1), (2, 2) AS t(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t2 AS +WITH v as ( + SELECT c1 + c1 c3 FROM t1 +) +SELECT SUM(c3) s FROM v +-- !query schema +struct<> +-- !query output + + + +-- !query +CACHE TABLE cache_table +WITH +t2 AS (SELECT 1) +SELECT * FROM t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM cache_table +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +EXPLAIN EXTENDED SELECT * FROM cache_table +-- !query schema +struct +-- !query output +== Parsed Logical Plan == +'Project [*] ++- 'UnresolvedRelation [cache_table], [], false + +== Analyzed Logical Plan == +1: int +Project [1#x] ++- SubqueryAlias cache_table + +- View (`cache_table`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias t2 + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [1#x] + +- SubqueryAlias t2 + +- CTERelationRef xxxx, true, [1#x], false, false, 1 + +== Optimized Logical Plan == +InMemoryRelation [1#x], StorageLevel(disk, memory, deserialized, 1 replicas) + +- *Project [1 AS 1#x] + +- *Scan OneRowRelation[] + +== Physical Plan == +Scan In-memory table cache_table [1#x] + +- InMemoryRelation [1#x], StorageLevel(disk, memory, deserialized, 1 replicas) + +- *Project [1 AS 1#x] + +- *Scan OneRowRelation[] + + +-- !query +CACHE TABLE cache_nested_cte_table +WITH +v AS ( + SELECT c1 * c2 c3 from t1 +) +SELECT SUM(c3) FROM v +EXCEPT +SELECT s FROM t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM cache_nested_cte_table +-- !query schema +struct +-- !query output +5 + + +-- !query +EXPLAIN EXTENDED SELECT * FROM cache_nested_cte_table +-- !query schema +struct +-- !query output +== Parsed Logical Plan == +'Project [*] ++- 'UnresolvedRelation [cache_nested_cte_table], [], false + +== Analyzed Logical Plan == +sum(c3): bigint +Project [sum(c3)#xL] ++- SubqueryAlias cache_nested_cte_table + +- View (`cache_nested_cte_table`, [sum(c3)#xL]) + +- Project [cast(sum(c3)#xL as bigint) AS sum(c3)#xL] + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias v + : +- Project [(c1#x * c2#x) AS c3#x] + : +- 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] + : +- Project [c1#x, c2#x] + : +- SubqueryAlias t + : +- LocalRelation [c1#x, c2#x] + +- Except false + :- Aggregate [sum(c3#x) AS sum(c3)#xL] + : +- SubqueryAlias v + : +- CTERelationRef xxxx, true, [c3#x], false, false + +- Project [s#xL] + +- SubqueryAlias t2 + +- View (`t2`, [s#xL]) + +- Project [cast(s#xL as bigint) AS s#xL] + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias v + : +- Project [(c1#x + c1#x) AS c3#x] + : +- 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] + : +- Project [c1#x, c2#x] + : +- SubqueryAlias t + : +- LocalRelation [c1#x, c2#x] + +- Aggregate [sum(c3#x) AS s#xL] + +- SubqueryAlias v + +- CTERelationRef xxxx, true, [c3#x], false, false + +== Optimized Logical Plan == +InMemoryRelation [sum(c3)#xL], StorageLevel(disk, memory, deserialized, 1 replicas) + +- AdaptiveSparkPlan isFinalPlan=true + +- == Final Plan == + ResultQueryStage 1 + +- *BroadcastHashJoin [coalesce(sum(c3)#xL, 0), isnull(sum(c3)#xL)], [coalesce(s#xL, 0), isnull(s#xL)], LeftAnti, BuildRight, false + :- *Project [ReusedSubquery Subquery subquery#x, [id=#x].sum(c3) AS sum(c3)#xL] + : : +- ReusedSubquery Subquery subquery#x, [id=#x] + : +- *Scan OneRowRelation[] + +- BroadcastQueryStage 0 + +- BroadcastExchange HashedRelationBroadcastMode(List(coalesce(input[0, bigint, true], 0), isnull(input[0, bigint, true])),false), [plan_id=x] + +- *Project [Subquery subquery#x, [id=#x].s AS s#xL] + : +- Subquery subquery#x, [id=#x] + : +- AdaptiveSparkPlan isFinalPlan=true + +- == Final Plan == + ResultQueryStage 1 + +- *Project [named_struct(sum(c3), sum(c3)#xL, s, s#xL) AS mergedValue#x] + +- *HashAggregate(keys=[], functions=[sum(c3#x), sum(c3#x)], output=[sum(c3)#xL, s#xL]) + +- ShuffleQueryStage 0 + +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- *HashAggregate(keys=[], functions=[partial_sum(c3#x), partial_sum(c3#x)], output=[sum#xL, sum#xL]) + +- *Project [(c1#x * c2#x) AS c3#x, (c1#x + c1#x) AS c3#x] + +- *LocalTableScan [c1#x, c2#x] + +- == Initial Plan == + Project [named_struct(sum(c3), sum(c3)#xL, s, s#xL) AS mergedValue#x] + +- HashAggregate(keys=[], functions=[sum(c3#x), sum(c3#x)], output=[sum(c3)#xL, s#xL]) + +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- HashAggregate(keys=[], functions=[partial_sum(c3#x), partial_sum(c3#x)], output=[sum#xL, sum#xL]) + +- Project [(c1#x * c2#x) AS c3#x, (c1#x + c1#x) AS c3#x] + +- LocalTableScan [c1#x, c2#x] + +- *Scan OneRowRelation[] + +- == Initial Plan == + BroadcastHashJoin [coalesce(sum(c3)#xL, 0), isnull(sum(c3)#xL)], [coalesce(s#xL, 0), isnull(s#xL)], LeftAnti, BuildRight, false + :- Project [Subquery subquery#x, [id=#x].sum(c3) AS sum(c3)#xL] + : : +- Subquery subquery#x, [id=#x] + : : +- AdaptiveSparkPlan isFinalPlan=false + : : +- Project [named_struct(sum(c3), sum(c3)#xL, s, s#xL) AS mergedValue#x] + : : +- HashAggregate(keys=[], functions=[sum(c3#x), sum(c3#x)], output=[sum(c3)#xL, s#xL]) + : : +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + : : +- HashAggregate(keys=[], functions=[partial_sum(c3#x), partial_sum(c3#x)], output=[sum#xL, sum#xL]) + : : +- Project [(c1#x * c2#x) AS c3#x, (c1#x + c1#x) AS c3#x] + : : +- LocalTableScan [c1#x, c2#x] + : +- Scan OneRowRelation[] + +- BroadcastExchange HashedRelationBroadcastMode(List(coalesce(input[0, bigint, true], 0), isnull(input[0, bigint, true])),false), [plan_id=x] + +- Project [Subquery subquery#x, [id=#x].s AS s#xL] + : +- Subquery subquery#x, [id=#x] + : +- AdaptiveSparkPlan isFinalPlan=true + +- == Final Plan == + ResultQueryStage 1 + +- *Project [named_struct(sum(c3), sum(c3)#xL, s, s#xL) AS mergedValue#x] + +- *HashAggregate(keys=[], functions=[sum(c3#x), sum(c3#x)], output=[sum(c3)#xL, s#xL]) + +- ShuffleQueryStage 0 + +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- *HashAggregate(keys=[], functions=[partial_sum(c3#x), partial_sum(c3#x)], output=[sum#xL, sum#xL]) + +- *Project [(c1#x * c2#x) AS c3#x, (c1#x + c1#x) AS c3#x] + +- *LocalTableScan [c1#x, c2#x] + +- == Initial Plan == + Project [named_struct(sum(c3), sum(c3)#xL, s, s#xL) AS mergedValue#x] + +- HashAggregate(keys=[], functions=[sum(c3#x), sum(c3#x)], output=[sum(c3)#xL, s#xL]) + +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- HashAggregate(keys=[], functions=[partial_sum(c3#x), partial_sum(c3#x)], output=[sum#xL, sum#xL]) + +- Project [(c1#x * c2#x) AS c3#x, (c1#x + c1#x) AS c3#x] + +- LocalTableScan [c1#x, c2#x] + +- Scan OneRowRelation[] + +== Physical Plan == +AdaptiveSparkPlan isFinalPlan=false ++- Scan In-memory table cache_nested_cte_table [sum(c3)#xL] + +- InMemoryRelation [sum(c3)#xL], StorageLevel(disk, memory, deserialized, 1 replicas) + +- AdaptiveSparkPlan isFinalPlan=true + +- == Final Plan == + ResultQueryStage 1 + +- *BroadcastHashJoin [coalesce(sum(c3)#xL, 0), isnull(sum(c3)#xL)], [coalesce(s#xL, 0), isnull(s#xL)], LeftAnti, BuildRight, false + :- *Project [ReusedSubquery Subquery subquery#x, [id=#x].sum(c3) AS sum(c3)#xL] + : : +- ReusedSubquery Subquery subquery#x, [id=#x] + : +- *Scan OneRowRelation[] + +- BroadcastQueryStage 0 + +- BroadcastExchange HashedRelationBroadcastMode(List(coalesce(input[0, bigint, true], 0), isnull(input[0, bigint, true])),false), [plan_id=x] + +- *Project [Subquery subquery#x, [id=#x].s AS s#xL] + : +- Subquery subquery#x, [id=#x] + : +- AdaptiveSparkPlan isFinalPlan=true + +- == Final Plan == + ResultQueryStage 1 + +- *Project [named_struct(sum(c3), sum(c3)#xL, s, s#xL) AS mergedValue#x] + +- *HashAggregate(keys=[], functions=[sum(c3#x), sum(c3#x)], output=[sum(c3)#xL, s#xL]) + +- ShuffleQueryStage 0 + +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- *HashAggregate(keys=[], functions=[partial_sum(c3#x), partial_sum(c3#x)], output=[sum#xL, sum#xL]) + +- *Project [(c1#x * c2#x) AS c3#x, (c1#x + c1#x) AS c3#x] + +- *LocalTableScan [c1#x, c2#x] + +- == Initial Plan == + Project [named_struct(sum(c3), sum(c3)#xL, s, s#xL) AS mergedValue#x] + +- HashAggregate(keys=[], functions=[sum(c3#x), sum(c3#x)], output=[sum(c3)#xL, s#xL]) + +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- HashAggregate(keys=[], functions=[partial_sum(c3#x), partial_sum(c3#x)], output=[sum#xL, sum#xL]) + +- Project [(c1#x * c2#x) AS c3#x, (c1#x + c1#x) AS c3#x] + +- LocalTableScan [c1#x, c2#x] + +- *Scan OneRowRelation[] + +- == Initial Plan == + BroadcastHashJoin [coalesce(sum(c3)#xL, 0), isnull(sum(c3)#xL)], [coalesce(s#xL, 0), isnull(s#xL)], LeftAnti, BuildRight, false + :- Project [Subquery subquery#x, [id=#x].sum(c3) AS sum(c3)#xL] + : : +- Subquery subquery#x, [id=#x] + : : +- AdaptiveSparkPlan isFinalPlan=false + : : +- Project [named_struct(sum(c3), sum(c3)#xL, s, s#xL) AS mergedValue#x] + : : +- HashAggregate(keys=[], functions=[sum(c3#x), sum(c3#x)], output=[sum(c3)#xL, s#xL]) + : : +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + : : +- HashAggregate(keys=[], functions=[partial_sum(c3#x), partial_sum(c3#x)], output=[sum#xL, sum#xL]) + : : +- Project [(c1#x * c2#x) AS c3#x, (c1#x + c1#x) AS c3#x] + : : +- LocalTableScan [c1#x, c2#x] + : +- Scan OneRowRelation[] + +- BroadcastExchange HashedRelationBroadcastMode(List(coalesce(input[0, bigint, true], 0), isnull(input[0, bigint, true])),false), [plan_id=x] + +- Project [Subquery subquery#x, [id=#x].s AS s#xL] + : +- Subquery subquery#x, [id=#x] + : +- AdaptiveSparkPlan isFinalPlan=true + +- == Final Plan == + ResultQueryStage 1 + +- *Project [named_struct(sum(c3), sum(c3)#xL, s, s#xL) AS mergedValue#x] + +- *HashAggregate(keys=[], functions=[sum(c3#x), sum(c3#x)], output=[sum(c3)#xL, s#xL]) + +- ShuffleQueryStage 0 + +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- *HashAggregate(keys=[], functions=[partial_sum(c3#x), partial_sum(c3#x)], output=[sum#xL, sum#xL]) + +- *Project [(c1#x * c2#x) AS c3#x, (c1#x + c1#x) AS c3#x] + +- *LocalTableScan [c1#x, c2#x] + +- == Initial Plan == + Project [named_struct(sum(c3), sum(c3)#xL, s, s#xL) AS mergedValue#x] + +- HashAggregate(keys=[], functions=[sum(c3#x), sum(c3#x)], output=[sum(c3)#xL, s#xL]) + +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- HashAggregate(keys=[], functions=[partial_sum(c3#x), partial_sum(c3#x)], output=[sum#xL, sum#xL]) + +- Project [(c1#x * c2#x) AS c3#x, (c1#x + c1#x) AS c3#x] + +- LocalTableScan [c1#x, c2#x] + +- Scan OneRowRelation[] + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t2 +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out index 536fc6c4ea63..f38474db4af8 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out @@ -341,68 +341,16 @@ WITH RECURSIVE t(n) MAX RECURSION LEVEL 100 AS ( ) SELECT * FROM t LIMIT ALL -- !query schema -struct +struct<> -- !query output -1 -10 -11 -12 -13 -14 -15 -16 -17 -18 -19 -2 -20 -21 -22 -23 -24 -25 -26 -27 -28 -29 -3 -30 -31 -32 -33 -34 -35 -36 -37 -38 -39 -4 -40 -41 -42 -43 -44 -45 -46 -47 -48 -49 -5 -50 -51 -52 -53 -54 -55 -56 -57 -58 -59 -6 -60 -7 -8 -9 +org.apache.spark.SparkException +{ + "errorClass" : "RECURSION_ROW_LIMIT_EXCEEDED", + "sqlState" : "42836", + "messageParameters" : { + "rowLimit" : "50" + } +} -- !query @@ -413,68 +361,16 @@ WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( ) SELECT * FROM t LIMIT ALL -- !query schema -struct +struct<> -- !query output -1 -10 -11 -12 -13 -14 -15 -16 -17 -18 -19 -2 -20 -21 -22 -23 -24 -25 -26 -27 -28 -29 -3 -30 -31 -32 -33 -34 -35 -36 -37 -38 -39 -4 -40 -41 -42 -43 -44 -45 -46 -47 -48 -49 -5 -50 -51 -52 -53 -54 -55 -56 -57 -58 -59 -6 -60 -7 -8 -9 +org.apache.spark.SparkException +{ + "errorClass" : "RECURSION_ROW_LIMIT_EXCEEDED", + "sqlState" : "42836", + "messageParameters" : { + "rowLimit" : "50" + } +} -- !query @@ -505,128 +401,16 @@ WITH RECURSIVE t MAX RECURSION LEVEL 100 AS ( ) (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t LIMIT ALL) -- !query schema -struct +struct<> -- !query output -1 -1 -10 -10 -11 -11 -12 -12 -13 -13 -14 -14 -15 -15 -16 -16 -17 -17 -18 -18 -19 -19 -2 -2 -20 -20 -21 -21 -22 -22 -23 -23 -24 -24 -25 -25 -26 -26 -27 -27 -28 -28 -29 -29 -3 -3 -30 -30 -31 -31 -32 -32 -33 -33 -34 -34 -35 -35 -36 -36 -37 -37 -38 -38 -39 -39 -4 -4 -40 -40 -41 -41 -42 -42 -43 -43 -44 -44 -45 -45 -46 -46 -47 -47 -48 -48 -49 -49 -5 -5 -50 -50 -51 -51 -52 -52 -53 -53 -54 -54 -55 -55 -56 -56 -57 -57 -58 -58 -59 -59 -6 -6 -60 -60 -7 -7 -8 -8 -9 -9 +org.apache.spark.SparkException +{ + "errorClass" : "RECURSION_ROW_LIMIT_EXCEEDED", + "sqlState" : "42836", + "messageParameters" : { + "rowLimit" : "50" + } +} -- !query