From 643b567c5fd1943ec5fb6ee9e576a10792c4b2c2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 9 Jun 2022 01:26:41 -0700 Subject: [PATCH] Revert "[SPARK-37670][SQL] Support predicate pushdown and column pruning for de-duped CTEs" This reverts commit 1a356858577aa13012f56fdd4b41f8ea6fe230ce. --- .../catalyst/analysis/CTESubstitution.scala | 30 +-- .../sql/catalyst/analysis/CheckAnalysis.scala | 8 +- .../sql/catalyst/optimizer/InlineCTE.scala | 56 ++--- .../sql/catalyst/optimizer/Optimizer.scala | 61 ++--- ...wnPredicatesAndPruneColumnsForCTEDef.scala | 175 ------------- .../ReplaceCTERefWithRepartition.scala | 84 ------- .../spark/sql/catalyst/plans/QueryPlan.scala | 8 - .../plans/logical/basicLogicalOperators.scala | 9 +- .../sql/catalyst/analysis/AnalysisTest.scala | 3 +- .../optimizer/ComputeCurrentTimeSuite.scala | 2 +- .../spark/sql/execution/QueryExecution.scala | 23 +- .../spark/sql/execution/SparkOptimizer.scala | 3 +- .../spark/sql/execution/SparkPlanner.scala | 1 + .../spark/sql/execution/SparkStrategies.scala | 31 +++ .../adaptive/AdaptiveSparkPlanExec.scala | 7 +- .../scalar-subquery-select.sql | 42 ---- .../scalar-subquery-select.sql.out | 103 +------- .../q23a.sf100/explain.txt | 160 ++++++------ .../q23b.sf100/explain.txt | 178 ++++++------- .../approved-plans-v1_4/q4/explain.txt | 238 +++++++++--------- .../approved-plans-v1_4/q5/explain.txt | 212 ++++++++-------- .../org/apache/spark/sql/CTEInlineSuite.scala | 229 +---------------- .../org/apache/spark/sql/SQLQuerySuite.scala | 15 -- 23 files changed, 523 insertions(+), 1155 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceCTERefWithRepartition.scala 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 5912696101fb3..a67d85d8433fc 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 @@ -66,13 +66,13 @@ object CTESubstitution extends Rule[LogicalPlan] { if (cteDefs.isEmpty) { substituted } else if (substituted eq lastSubstituted.get) { - WithCTE(substituted, cteDefs.sortBy(_.id).toSeq) + WithCTE(substituted, cteDefs.toSeq) } else { var done = false substituted.resolveOperatorsWithPruning(_ => !done) { case p if p eq lastSubstituted.get => done = true - WithCTE(p, cteDefs.sortBy(_.id).toSeq) + WithCTE(p, cteDefs.toSeq) } } } @@ -200,7 +200,6 @@ object CTESubstitution extends Rule[LogicalPlan] { cteDefs: mutable.ArrayBuffer[CTERelationDef]): Seq[(String, CTERelationDef)] = { val resolvedCTERelations = new mutable.ArrayBuffer[(String, CTERelationDef)](relations.size) for ((name, relation) <- relations) { - val lastCTEDefCount = cteDefs.length val innerCTEResolved = if (isLegacy) { // In legacy mode, outer CTE relations take precedence. Here we don't resolve the inner // `With` nodes, later we will substitute `UnresolvedRelation`s with outer CTE relations. @@ -209,33 +208,8 @@ object CTESubstitution extends Rule[LogicalPlan] { } else { // A CTE definition might contain an inner CTE that has a higher priority, so traverse and // substitute CTE defined in `relation` first. - // NOTE: we must call `traverseAndSubstituteCTE` before `substituteCTE`, as the relations - // in the inner CTE have higher priority over the relations in the outer CTE when resolving - // inner CTE relations. For example: - // WITH t1 AS (SELECT 1) - // t2 AS ( - // WITH t1 AS (SELECT 2) - // WITH t3 AS (SELECT * FROM t1) - // ) - // t3 should resolve the t1 to `SELECT 2` instead of `SELECT 1`. traverseAndSubstituteCTE(relation, isCommand, cteDefs)._1 } - - if (cteDefs.length > lastCTEDefCount) { - // We have added more CTE relations to the `cteDefs` from the inner CTE, and these relations - // should also be substituted with `resolvedCTERelations` as inner CTE relation can refer to - // outer CTE relation. For example: - // WITH t1 AS (SELECT 1) - // t2 AS ( - // WITH t3 AS (SELECT * FROM t1) - // ) - for (i <- lastCTEDefCount until cteDefs.length) { - val substituted = - substituteCTE(cteDefs(i).child, isLegacy || isCommand, resolvedCTERelations.toSeq) - cteDefs(i) = cteDefs(i).copy(child = substituted) - } - } - // CTE definition can reference a previous one val substituted = substituteCTE(innerCTEResolved, isLegacy || isCommand, resolvedCTERelations.toSeq) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index caf9514ab9488..d38c1bab0e485 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression -import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, DecorrelateInnerQuery, InlineCTE} +import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, DecorrelateInnerQuery} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, TypeUtils} @@ -90,10 +90,8 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { def checkAnalysis(plan: LogicalPlan): Unit = { // We transform up and order the rules so as to catch the first possible failure instead - // of the result of cascading resolution failures. Inline all CTEs in the plan to help check - // query plan structures in subqueries. - val inlineCTE = InlineCTE(alwaysInline = true) - inlineCTE(plan).foreachUp { + // of the result of cascading resolution failures. + plan.foreachUp { case p if p.analyzed => // Skip already analyzed sub-plans diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala index e0f396dde3850..6bcbc9f821de6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala @@ -28,37 +28,26 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.{CTE, PLAN_EXPRESSION} /** * Inlines CTE definitions into corresponding references if either of the conditions satisfies: - * 1. The CTE definition does not contain any non-deterministic expressions or contains attribute - * references to an outer query. If this CTE definition references another CTE definition that - * has non-deterministic expressions, it is still OK to inline the current CTE definition. + * 1. The CTE definition does not contain any non-deterministic expressions. If this CTE + * definition references another CTE definition that has non-deterministic expressions, it + * is still OK to inline the current CTE definition. * 2. The CTE definition is only referenced once throughout the main query and all the subqueries. * - * CTE definitions that appear in subqueries and are not inlined will be pulled up to the main - * query level. - * - * @param alwaysInline if true, inline all CTEs in the query plan. + * In addition, due to the complexity of correlated subqueries, all CTE references in correlated + * subqueries are inlined regardless of the conditions above. */ -case class InlineCTE(alwaysInline: Boolean = false) extends Rule[LogicalPlan] { - +object InlineCTE extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = { if (!plan.isInstanceOf[Subquery] && plan.containsPattern(CTE)) { val cteMap = mutable.HashMap.empty[Long, (CTERelationDef, Int)] buildCTEMap(plan, cteMap) - val notInlined = mutable.ArrayBuffer.empty[CTERelationDef] - val inlined = inlineCTE(plan, cteMap, notInlined) - // CTEs in SQL Commands have been inlined by `CTESubstitution` already, so it is safe to add - // WithCTE as top node here. - if (notInlined.isEmpty) { - inlined - } else { - WithCTE(inlined, notInlined.toSeq) - } + inlineCTE(plan, cteMap, forceInline = false) } else { plan } } - private def shouldInline(cteDef: CTERelationDef, refCount: Int): Boolean = alwaysInline || { + private def shouldInline(cteDef: CTERelationDef, refCount: Int): Boolean = { // We do not need to check enclosed `CTERelationRef`s for `deterministic` or `OuterReference`, // because: // 1) It is fine to inline a CTE if it references another CTE that is non-deterministic; @@ -104,24 +93,25 @@ case class InlineCTE(alwaysInline: Boolean = false) extends Rule[LogicalPlan] { private def inlineCTE( plan: LogicalPlan, cteMap: mutable.HashMap[Long, (CTERelationDef, Int)], - notInlined: mutable.ArrayBuffer[CTERelationDef]): LogicalPlan = { - plan match { + forceInline: Boolean): LogicalPlan = { + val (stripped, notInlined) = plan match { case WithCTE(child, cteDefs) => + val notInlined = mutable.ArrayBuffer.empty[CTERelationDef] cteDefs.foreach { cteDef => val (cte, refCount) = cteMap(cteDef.id) if (refCount > 0) { - val inlined = cte.copy(child = inlineCTE(cte.child, cteMap, notInlined)) + val inlined = cte.copy(child = inlineCTE(cte.child, cteMap, forceInline)) cteMap.update(cteDef.id, (inlined, refCount)) - if (!shouldInline(inlined, refCount)) { + if (!forceInline && !shouldInline(inlined, refCount)) { notInlined.append(inlined) } } } - inlineCTE(child, cteMap, notInlined) + (inlineCTE(child, cteMap, forceInline), notInlined.toSeq) case ref: CTERelationRef => val (cteDef, refCount) = cteMap(ref.cteId) - if (shouldInline(cteDef, refCount)) { + val newRef = if (forceInline || shouldInline(cteDef, refCount)) { if (ref.outputSet == cteDef.outputSet) { cteDef.child } else { @@ -135,16 +125,24 @@ case class InlineCTE(alwaysInline: Boolean = false) extends Rule[LogicalPlan] { } else { ref } + (newRef, Seq.empty) case _ if plan.containsPattern(CTE) => - plan - .withNewChildren(plan.children.map(child => inlineCTE(child, cteMap, notInlined))) + val newPlan = plan + .withNewChildren(plan.children.map(child => inlineCTE(child, cteMap, forceInline))) .transformExpressionsWithPruning(_.containsAllPatterns(PLAN_EXPRESSION, CTE)) { case e: SubqueryExpression => - e.withNewPlan(inlineCTE(e.plan, cteMap, notInlined)) + e.withNewPlan(inlineCTE(e.plan, cteMap, forceInline = e.isCorrelated)) } + (newPlan, Seq.empty) - case _ => plan + case _ => (plan, Seq.empty) + } + + if (notInlined.isEmpty) { + stripped + } else { + WithCTE(stripped, notInlined) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 4fe00ef0eedf5..a92c1d09e957a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -126,8 +126,7 @@ abstract class Optimizer(catalogManager: CatalogManager) OptimizeUpdateFields, SimplifyExtractValueOps, OptimizeCsvJsonExprs, - CombineConcats, - PushdownPredicatesAndPruneColumnsForCTEDef) ++ + CombineConcats) ++ extendedOperatorOptimizationRules val operatorOptimizationBatch: Seq[Batch] = { @@ -146,7 +145,21 @@ abstract class Optimizer(catalogManager: CatalogManager) } val batches = (Batch("Eliminate Distinct", Once, EliminateDistinct) :: - Batch("Finish Analysis", Once, FinishAnalysis) :: + // Technically some of the rules in Finish Analysis are not optimizer rules and belong more + // in the analyzer, because they are needed for correctness (e.g. ComputeCurrentTime). + // However, because we also use the analyzer to canonicalized queries (for view definition), + // we do not eliminate subqueries or compute current time in the analyzer. + Batch("Finish Analysis", Once, + EliminateResolvedHint, + EliminateSubqueryAliases, + EliminateView, + InlineCTE, + ReplaceExpressions, + RewriteNonCorrelatedExists, + PullOutGroupingExpressions, + ComputeCurrentTime, + ReplaceCurrentLike(catalogManager), + SpecialDatetimeValues) :: ////////////////////////////////////////////////////////////////////////////////////////// // Optimizer rules start here ////////////////////////////////////////////////////////////////////////////////////////// @@ -155,8 +168,6 @@ abstract class Optimizer(catalogManager: CatalogManager) // extra operators between two adjacent Union operators. // - Call CombineUnions again in Batch("Operator Optimizations"), // since the other rules might make two separate Unions operators adjacent. - Batch("Inline CTE", Once, - InlineCTE()) :: Batch("Union", Once, RemoveNoopOperators, CombineUnions, @@ -193,7 +204,6 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveLiteralFromGroupExpressions, RemoveRepetitionFromGroupExpressions) :: Nil ++ operatorOptimizationBatch) :+ - Batch("Clean Up Temporary CTE Info", Once, CleanUpTempCTEInfo) :+ // This batch rewrites plans after the operator optimization and // before any batches that depend on stats. Batch("Pre CBO Rules", Once, preCBORules: _*) :+ @@ -250,7 +260,14 @@ abstract class Optimizer(catalogManager: CatalogManager) * (defaultBatches - (excludedRules - nonExcludableRules)). */ def nonExcludableRules: Seq[String] = - FinishAnalysis.ruleName :: + EliminateDistinct.ruleName :: + EliminateResolvedHint.ruleName :: + EliminateSubqueryAliases.ruleName :: + EliminateView.ruleName :: + ReplaceExpressions.ruleName :: + ComputeCurrentTime.ruleName :: + SpecialDatetimeValues.ruleName :: + ReplaceCurrentLike(catalogManager).ruleName :: RewriteDistinctAggregates.ruleName :: ReplaceDeduplicateWithAggregate.ruleName :: ReplaceIntersectWithSemiJoin.ruleName :: @@ -264,37 +281,9 @@ abstract class Optimizer(catalogManager: CatalogManager) RewritePredicateSubquery.ruleName :: NormalizeFloatingNumbers.ruleName :: ReplaceUpdateFieldsExpression.ruleName :: + PullOutGroupingExpressions.ruleName :: RewriteLateralSubquery.ruleName :: Nil - /** - * Apply finish-analysis rules for the entire plan including all subqueries. - */ - object FinishAnalysis extends Rule[LogicalPlan] { - // Technically some of the rules in Finish Analysis are not optimizer rules and belong more - // in the analyzer, because they are needed for correctness (e.g. ComputeCurrentTime). - // However, because we also use the analyzer to canonicalized queries (for view definition), - // we do not eliminate subqueries or compute current time in the analyzer. - private val rules = Seq( - EliminateResolvedHint, - EliminateSubqueryAliases, - EliminateView, - ReplaceExpressions, - RewriteNonCorrelatedExists, - PullOutGroupingExpressions, - ComputeCurrentTime, - ReplaceCurrentLike(catalogManager), - SpecialDatetimeValues) - - override def apply(plan: LogicalPlan): LogicalPlan = { - rules.foldLeft(plan) { case (sp, rule) => rule.apply(sp) } - .transformAllExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { - case s: SubqueryExpression => - val Subquery(newPlan, _) = apply(Subquery.fromExpression(s)) - s.withNewPlan(newPlan) - } - } - } - /** * Optimize all the subqueries inside expression. */ 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 deleted file mode 100644 index ab9f20edb0bb9..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala +++ /dev/null @@ -1,175 +0,0 @@ -/* - * 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.optimizer - -import scala.collection.mutable - -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, Literal, Or, SubqueryExpression} -import org.apache.spark.sql.catalyst.planning.ScanOperation -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.TreePattern.CTE - -/** - * Infer predicates and column pruning for [[CTERelationDef]] from its reference points, and push - * the disjunctive predicates as well as the union of attributes down the CTE plan. - */ -object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { - - // CTE_id - (CTE_definition, precedence, predicates_to_push_down, attributes_to_prune) - private type CTEMap = mutable.HashMap[Long, (CTERelationDef, Int, Seq[Expression], AttributeSet)] - - override def apply(plan: LogicalPlan): LogicalPlan = { - if (!plan.isInstanceOf[Subquery] && plan.containsPattern(CTE)) { - val cteMap = new CTEMap - gatherPredicatesAndAttributes(plan, cteMap) - pushdownPredicatesAndAttributes(plan, cteMap) - } else { - plan - } - } - - private def restoreCTEDefAttrs( - input: Seq[Expression], - mapping: Map[Attribute, Expression]): Seq[Expression] = { - input.map(e => e.transform { - case a: Attribute => - mapping.keys.find(_.semanticEquals(a)).map(mapping).getOrElse(a) - }) - } - - /** - * Gather all the predicates and referenced attributes on different points of CTE references - * using pattern `ScanOperation` (which takes care of determinism) and combine those predicates - * and attributes that belong to the same CTE definition. - * For the same CTE definition, if any of its references does not have predicates, the combined - * predicate will be a TRUE literal, which means there will be no predicate push-down. - */ - private def gatherPredicatesAndAttributes(plan: LogicalPlan, cteMap: CTEMap): Unit = { - plan match { - case WithCTE(child, cteDefs) => - cteDefs.zipWithIndex.foreach { case (cteDef, precedence) => - gatherPredicatesAndAttributes(cteDef.child, cteMap) - cteMap.put(cteDef.id, (cteDef, precedence, Seq.empty, AttributeSet.empty)) - } - gatherPredicatesAndAttributes(child, cteMap) - - case ScanOperation(projects, predicates, ref: CTERelationRef) => - val (cteDef, precedence, preds, attrs) = cteMap(ref.cteId) - val attrMapping = ref.output.zip(cteDef.output).map{ case (r, d) => r -> d }.toMap - val newPredicates = if (isTruePredicate(preds)) { - preds - } else { - // Make sure we only push down predicates that do not contain forward CTE references. - val filteredPredicates = restoreCTEDefAttrs(predicates.filter(_.find { - case s: SubqueryExpression => s.plan.find { - case r: CTERelationRef => - // If the ref's ID does not exist in the map or if ref's corresponding precedence - // is bigger than that of the current CTE we are pushing predicates for, it - // indicates a forward reference and we should exclude this predicate. - !cteMap.contains(r.cteId) || cteMap(r.cteId)._2 >= precedence - case _ => false - }.nonEmpty - case _ => false - }.isEmpty), attrMapping).filter(_.references.forall(cteDef.outputSet.contains)) - if (filteredPredicates.isEmpty) { - Seq(Literal.TrueLiteral) - } else { - preds :+ filteredPredicates.reduce(And) - } - } - val newAttributes = attrs ++ - AttributeSet(restoreCTEDefAttrs(projects.flatMap(_.references), attrMapping)) ++ - AttributeSet(restoreCTEDefAttrs(predicates.flatMap(_.references), attrMapping)) - - cteMap.update(ref.cteId, (cteDef, precedence, newPredicates, newAttributes)) - plan.subqueriesAll.foreach(s => gatherPredicatesAndAttributes(s, cteMap)) - - case _ => - plan.children.foreach(c => gatherPredicatesAndAttributes(c, cteMap)) - plan.subqueries.foreach(s => gatherPredicatesAndAttributes(s, cteMap)) - } - } - - /** - * Push down the combined predicate and attribute references to each CTE definition plan. - * - * In order to guarantee idempotency, we keep the predicates (if any) being pushed down by the - * last iteration of this rule in a temporary field of `CTERelationDef`, so that on the current - * iteration, we only push down predicates for a CTE def if there exists any new predicate that - * has not been pushed before. Also, since part of a new predicate might overlap with some - * existing predicate and it can be hard to extract only the non-overlapping part, we also keep - * the original CTE definition plan without any predicate push-down in that temporary field so - * that when we do a new predicate push-down, we can construct a new plan with all latest - * predicates over the original plan without having to figure out the exact predicate difference. - */ - private def pushdownPredicatesAndAttributes( - plan: LogicalPlan, - cteMap: CTEMap): LogicalPlan = plan.transformWithSubqueries { - 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) - if (!isTruePredicate(newPreds) && - newPreds.exists(newPred => !preds.exists(_.semanticEquals(newPred)))) { - val newCombinedPred = newPreds.reduce(Or) - val newChild = if (needsPruning(originalPlan, newAttrSet)) { - Project(newAttrSet.toSeq, originalPlan) - } else { - originalPlan - } - CTERelationDef(Filter(newCombinedPred, newChild), id, Some((originalPlan, newPreds))) - } else if (needsPruning(cteDef.child, newAttrSet)) { - CTERelationDef(Project(newAttrSet.toSeq, cteDef.child), id, Some((originalPlan, preds))) - } else { - cteDef - } - - case cteRef @ CTERelationRef(cteId, _, output, _) => - val (cteDef, _, _, newAttrSet) = cteMap(cteId) - if (newAttrSet.size < output.size) { - val indices = newAttrSet.toSeq.map(cteDef.output.indexOf) - val newOutput = indices.map(output) - cteRef.copy(output = newOutput) - } else { - // Do not change the order of output columns if no column is pruned, in which case there - // might be no Project and the order is important. - cteRef - } - } - - private def isTruePredicate(predicates: Seq[Expression]): Boolean = { - predicates.length == 1 && predicates.head == Literal.TrueLiteral - } - - private def needsPruning(sourcePlan: LogicalPlan, attributeSet: AttributeSet): Boolean = { - attributeSet.size < sourcePlan.outputSet.size && attributeSet.subsetOf(sourcePlan.outputSet) - } -} - -/** - * Clean up temporary info from [[CTERelationDef]] nodes. This rule should be called after all - * iterations of [[PushdownPredicatesAndPruneColumnsForCTEDef]] are done. - */ -object CleanUpTempCTEInfo extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = - plan.transformWithPruning(_.containsPattern(CTE)) { - case cteDef @ CTERelationDef(_, _, Some(_)) => - cteDef.copy(originalPlanWithPredicates = None) - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceCTERefWithRepartition.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceCTERefWithRepartition.scala deleted file mode 100644 index e0d0417ce5161..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceCTERefWithRepartition.scala +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.optimizer - -import scala.collection.mutable - -import org.apache.spark.sql.catalyst.analysis.DeduplicateRelations -import org.apache.spark.sql.catalyst.expressions.{Alias, SubqueryExpression} -import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.TreePattern.{CTE, PLAN_EXPRESSION} - -/** - * Replaces CTE references that have not been previously inlined with [[Repartition]] operations - * which will then be planned as shuffles and reused across different reference points. - * - * Note that this rule should be called at the very end of the optimization phase to best guarantee - * that CTE repartition shuffles are reused. - */ -object ReplaceCTERefWithRepartition extends Rule[LogicalPlan] { - - override def apply(plan: LogicalPlan): LogicalPlan = plan match { - case _: Subquery => plan - case _ => - replaceWithRepartition(plan, mutable.HashMap.empty[Long, LogicalPlan]) - } - - private def replaceWithRepartition( - plan: LogicalPlan, - cteMap: mutable.HashMap[Long, LogicalPlan]): LogicalPlan = plan match { - case WithCTE(child, cteDefs) => - cteDefs.foreach { cteDef => - val inlined = replaceWithRepartition(cteDef.child, cteMap) - val withRepartition = if (inlined.isInstanceOf[RepartitionOperation]) { - // If the CTE definition plan itself is a repartition operation, we do not need to add an - // extra repartition shuffle. - inlined - } else { - Repartition(conf.numShufflePartitions, shuffle = true, inlined) - } - cteMap.put(cteDef.id, withRepartition) - } - replaceWithRepartition(child, cteMap) - - case ref: CTERelationRef => - val cteDefPlan = cteMap(ref.cteId) - if (ref.outputSet == cteDefPlan.outputSet) { - cteDefPlan - } else { - val ctePlan = DeduplicateRelations( - Join(cteDefPlan, cteDefPlan, Inner, None, JoinHint(None, None))).children(1) - val projectList = ref.output.zip(ctePlan.output).map { case (tgtAttr, srcAttr) => - Alias(srcAttr, tgtAttr.name)(exprId = tgtAttr.exprId) - } - Project(projectList, ctePlan) - } - - case _ if plan.containsPattern(CTE) => - plan - .withNewChildren(plan.children.map(c => replaceWithRepartition(c, cteMap))) - .transformExpressionsWithPruning(_.containsAllPatterns(PLAN_EXPRESSION, CTE)) { - case e: SubqueryExpression => - e.withNewPlan(replaceWithRepartition(e.plan, cteMap)) - } - - case _ => plan - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 9087916a4a369..73169262f7fdc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -441,14 +441,6 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] subqueries ++ subqueries.flatMap(_.subqueriesAll) } - /** - * This method is similar to the transform method, but also applies the given partial function - * also to all the plans in the subqueries of a node. This method is useful when we want - * to rewrite the whole plan, include its subqueries, in one go. - */ - def transformWithSubqueries(f: PartialFunction[PlanType, PlanType]): PlanType = - transformDownWithSubqueries(f) - /** * Returns a copy of this node where the given partial function has been recursively applied * first to the subqueries in this node's children, then this node's children, and finally 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 010722c034917..5c07d2ff678ac 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 @@ -657,15 +657,8 @@ case class UnresolvedWith( * A wrapper for CTE definition plan with a unique ID. * @param child The CTE definition query plan. * @param id The unique ID for this CTE definition. - * @param originalPlanWithPredicates The original query plan before predicate pushdown and the - * predicates that have been pushed down into `child`. This is - * a temporary field used by optimization rules for CTE predicate - * pushdown to help ensure rule idempotency. */ -case class CTERelationDef( - child: LogicalPlan, - id: Long = CTERelationDef.newId, - originalPlanWithPredicates: Option[(LogicalPlan, Seq[Expression])] = None) extends UnaryNode { +case class CTERelationDef(child: LogicalPlan, id: Long = CTERelationDef.newId) extends UnaryNode { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index f3b5b4fb64ff4..246538438c07a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -108,8 +108,7 @@ trait AnalysisTest extends PlanTest { case v: View if v.isTempViewStoringAnalyzedPlan => v.child } val actualPlan = if (inlineCTE) { - val inlineCTE = InlineCTE() - inlineCTE(transformed) + InlineCTE(transformed) } else { transformed } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala index 86461522f7469..739f70ab59acb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala @@ -137,7 +137,7 @@ class ComputeCurrentTimeSuite extends PlanTest { private def literals[T](plan: LogicalPlan): scala.collection.mutable.ArrayBuffer[T] = { val literals = new scala.collection.mutable.ArrayBuffer[T] - plan.transformWithSubqueries { case subQuery => + plan.transformDownWithSubqueries { case subQuery => subQuery.transformAllExpressions { case expression: Literal => literals += expression.value.asInstanceOf[T] expression diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 4a38f515c0371..4c5b0cc3de48c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -21,6 +21,8 @@ import java.io.{BufferedWriter, OutputStreamWriter} import java.util.UUID import java.util.concurrent.atomic.AtomicLong +import scala.collection.mutable + import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging @@ -30,7 +32,7 @@ import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.catalyst.expressions.codegen.ByteCodeStats import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, Command, CommandResult, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect, ReturnAnswer} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, Command, CommandResult, CreateTableAsSelect, CTERelationDef, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.{PlanChangeLogger, Rule} import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString @@ -62,6 +64,17 @@ class QueryExecution( // TODO: Move the planner an optimizer into here from SessionState. protected def planner = sparkSession.sessionState.planner + // The CTE map for the planner shared by the main query and all subqueries. + private val cteMap = mutable.HashMap.empty[Long, CTERelationDef] + + def withCteMap[T](f: => T): T = { + val old = QueryExecution.currentCteMap.get() + QueryExecution.currentCteMap.set(cteMap) + try f finally { + QueryExecution.currentCteMap.set(old) + } + } + def assertAnalyzed(): Unit = analyzed def assertSupported(): Unit = { @@ -134,7 +147,7 @@ class QueryExecution( private def assertOptimized(): Unit = optimizedPlan - lazy val sparkPlan: SparkPlan = { + lazy val sparkPlan: SparkPlan = withCteMap { // We need to materialize the optimizedPlan here because sparkPlan is also tracked under // the planning phase assertOptimized() @@ -147,7 +160,7 @@ class QueryExecution( // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. - lazy val executedPlan: SparkPlan = { + lazy val executedPlan: SparkPlan = withCteMap { // We need to materialize the optimizedPlan here, before tracking the planning phase, to ensure // that the optimization time is not counted as part of the planning phase. assertOptimized() @@ -481,4 +494,8 @@ object QueryExecution { val preparationRules = preparations(session, Option(InsertAdaptiveSparkPlan(context)), true) prepareForExecution(preparationRules, sparkPlan.clone()) } + + private val currentCteMap = new ThreadLocal[mutable.HashMap[Long, CTERelationDef]]() + + def cteMap: mutable.HashMap[Long, CTERelationDef] = currentCteMap.get() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index d746cb0f4d4a5..dc3ceb5c595d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -63,8 +63,7 @@ class SparkOptimizer( ColumnPruning, PushPredicateThroughNonJoin, RemoveNoopOperators) :+ - Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) :+ - Batch("Replace CTE with Repartition", Once, ReplaceCTERefWithRepartition) + Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) override def nonExcludableRules: Seq[String] = super.nonExcludableRules :+ ExtractPythonUDFFromJoinCondition.ruleName :+ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala index 6994aaf47dfba..32ac58f8353ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala @@ -44,6 +44,7 @@ class SparkPlanner(val session: SparkSession, val experimentalMethods: Experimen JoinSelection :: InMemoryScans :: SparkScripts :: + WithCTEStrategy :: BasicOperators :: Nil) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 9d7af6b5541fa..fc2898bf24711 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, JoinSelec import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.physical.RoundRobinPartitioning import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.aggregate.AggUtils @@ -620,6 +621,36 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } + /** + * Strategy to plan CTE relations left not inlined. + */ + object WithCTEStrategy extends Strategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case WithCTE(plan, cteDefs) => + val cteMap = QueryExecution.cteMap + cteDefs.foreach { cteDef => + cteMap.put(cteDef.id, cteDef) + } + planLater(plan) :: Nil + + case r: CTERelationRef => + val ctePlan = QueryExecution.cteMap(r.cteId).child + val projectList = r.output.zip(ctePlan.output).map { case (tgtAttr, srcAttr) => + Alias(srcAttr, tgtAttr.name)(exprId = tgtAttr.exprId) + } + val newPlan = Project(projectList, ctePlan) + // Plan CTE ref as a repartition shuffle so that all refs of the same CTE def will share + // an Exchange reuse at runtime. + // TODO create a new identity partitioning instead of using RoundRobinPartitioning. + exchange.ShuffleExchangeExec( + RoundRobinPartitioning(conf.numShufflePartitions), + planLater(newPlan), + REPARTITION_BY_COL) :: Nil + + case _ => Nil + } + } + object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case d: DataWritingCommand => DataWritingCommandExec(d, planLater(d.query)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index fe5b016d75c37..bf810f3f65877 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -138,7 +138,9 @@ case class AdaptiveSparkPlanExec( collapseCodegenStagesRule ) - private def optimizeQueryStage(plan: SparkPlan, isFinalStage: Boolean): SparkPlan = { + private def optimizeQueryStage( + plan: SparkPlan, + isFinalStage: Boolean): SparkPlan = context.qe.withCteMap { val optimized = queryStageOptimizerRules.foldLeft(plan) { case (latestPlan, rule) => val applied = rule.apply(latestPlan) val result = rule match { @@ -637,7 +639,8 @@ case class AdaptiveSparkPlanExec( /** * Re-optimize and run physical planning on the current logical plan based on the latest stats. */ - private def reOptimize(logicalPlan: LogicalPlan): (SparkPlan, LogicalPlan) = { + private def reOptimize( + logicalPlan: LogicalPlan): (SparkPlan, LogicalPlan) = context.qe.withCteMap { logicalPlan.invalidateStatsCache() val optimized = optimizer.execute(logicalPlan) val sparkPlan = context.session.sessionState.planner.plan(ReturnAnswer(optimized)).next() diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql index 4c80b268c20c3..a76a010722090 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql @@ -145,45 +145,3 @@ SELECT t1c, (SELECT t1c WHERE t1c = 8) FROM t1; SELECT t1c, t1d, (SELECT c + d FROM (SELECT t1c AS c, t1d AS d)) FROM t1; SELECT t1c, (SELECT SUM(c) FROM (SELECT t1c AS c)) FROM t1; SELECT t1a, (SELECT SUM(t2b) FROM t2 JOIN (SELECT t1a AS a) ON t2a = a) FROM t1; - --- CTE in correlated scalar subqueries -CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (0, 1), (1, 2) t1(c1, c2); -CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (0, 2), (0, 3) t2(c1, c2); - --- Single row subquery -SELECT c1, (WITH t AS (SELECT 1 AS a) SELECT a + c1 FROM t) FROM t1; --- Correlation in CTE. -SELECT c1, (WITH t AS (SELECT * FROM t2 WHERE c1 = t1.c1) SELECT SUM(c2) FROM t) FROM t1; --- Multiple CTE definitions. -SELECT c1, ( - WITH t3 AS (SELECT c1 + 1 AS c1, c2 + 1 AS c2 FROM t2), - t4 AS (SELECT * FROM t3 WHERE t1.c1 = c1) - SELECT SUM(c2) FROM t4 -) FROM t1; --- Multiple CTE references. -SELECT c1, ( - WITH t AS (SELECT * FROM t2) - SELECT SUM(c2) FROM (SELECT c1, c2 FROM t UNION SELECT c2, c1 FROM t) r(c1, c2) - WHERE c1 = t1.c1 -) FROM t1; --- Reference CTE in both the main query and the subquery. -WITH v AS (SELECT * FROM t2) -SELECT * FROM t1 WHERE c1 > ( - WITH t AS (SELECT * FROM t2) - SELECT COUNT(*) FROM v WHERE c1 = t1.c1 AND c1 > (SELECT SUM(c2) FROM t WHERE c1 = v.c1) -); --- Single row subquery that references CTE in the main query. -WITH t AS (SELECT 1 AS a) -SELECT c1, (SELECT a FROM t WHERE a = c1) FROM t1; --- Multiple CTE references with non-deterministic CTEs. -WITH -v1 AS (SELECT c1, c2, rand(0) c3 FROM t1), -v2 AS (SELECT c1, c2, rand(0) c4 FROM v1 WHERE c3 IN (SELECT c3 FROM v1)) -SELECT c1, ( - WITH v3 AS (SELECT c1, c2, rand(0) c5 FROM t2) - SELECT COUNT(*) FROM ( - SELECT * FROM v2 WHERE c1 > 0 - UNION SELECT * FROM v2 WHERE c2 > 0 - UNION SELECT * FROM v3 WHERE c2 > 0 - ) WHERE c1 = v1.c1 -) FROM v1; diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out index 3eb1c6ffba187..8fac940f8efd0 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 26 +-- Number of queries: 17 -- !query @@ -317,104 +317,3 @@ val1d NULL val1e 8 val1e 8 val1e 8 - - --- !query -CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (0, 1), (1, 2) t1(c1, c2) --- !query schema -struct<> --- !query output - - - --- !query -CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (0, 2), (0, 3) t2(c1, c2) --- !query schema -struct<> --- !query output - - - --- !query -SELECT c1, (WITH t AS (SELECT 1 AS a) SELECT a + c1 FROM t) FROM t1 --- !query schema -struct --- !query output -0 1 -1 2 - - --- !query -SELECT c1, (WITH t AS (SELECT * FROM t2 WHERE c1 = t1.c1) SELECT SUM(c2) FROM t) FROM t1 --- !query schema -struct --- !query output -0 5 -1 NULL - - --- !query -SELECT c1, ( - WITH t3 AS (SELECT c1 + 1 AS c1, c2 + 1 AS c2 FROM t2), - t4 AS (SELECT * FROM t3 WHERE t1.c1 = c1) - SELECT SUM(c2) FROM t4 -) FROM t1 --- !query schema -struct --- !query output -0 NULL -1 7 - - --- !query -SELECT c1, ( - WITH t AS (SELECT * FROM t2) - SELECT SUM(c2) FROM (SELECT c1, c2 FROM t UNION SELECT c2, c1 FROM t) r(c1, c2) - WHERE c1 = t1.c1 -) FROM t1 --- !query schema -struct --- !query output -0 5 -1 NULL - - --- !query -WITH v AS (SELECT * FROM t2) -SELECT * FROM t1 WHERE c1 > ( - WITH t AS (SELECT * FROM t2) - SELECT COUNT(*) FROM v WHERE c1 = t1.c1 AND c1 > (SELECT SUM(c2) FROM t WHERE c1 = v.c1) -) --- !query schema -struct --- !query output -1 2 - - --- !query -WITH t AS (SELECT 1 AS a) -SELECT c1, (SELECT a FROM t WHERE a = c1) FROM t1 --- !query schema -struct --- !query output -0 NULL -1 1 - - --- !query -WITH -v1 AS (SELECT c1, c2, rand(0) c3 FROM t1), -v2 AS (SELECT c1, c2, rand(0) c4 FROM v1 WHERE c3 IN (SELECT c3 FROM v1)) -SELECT c1, ( - WITH v3 AS (SELECT c1, c2, rand(0) c5 FROM t2) - SELECT COUNT(*) FROM ( - SELECT * FROM v2 WHERE c1 > 0 - UNION SELECT * FROM v2 WHERE c2 > 0 - UNION SELECT * FROM v3 WHERE c2 > 0 - ) WHERE c1 = v1.c1 -) FROM v1 --- !query schema -struct --- !query output -0 3 -1 1 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt index 76b0ff289c038..be706fee66776 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt @@ -360,19 +360,19 @@ Right keys [1]: [i_item_sk#14] Join condition: None (61) Project [codegen id : 25] -Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#17] +Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#47] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#14, i_item_desc#15] (62) HashAggregate [codegen id : 25] -Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#17] -Keys [3]: [_groupingexpression#17, i_item_sk#14, d_date#12] +Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#47] +Keys [3]: [_groupingexpression#47, i_item_sk#14, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] -Results [4]: [_groupingexpression#17, i_item_sk#14, d_date#12, count#19] +Results [4]: [_groupingexpression#47, i_item_sk#14, d_date#12, count#19] (63) HashAggregate [codegen id : 25] -Input [4]: [_groupingexpression#17, i_item_sk#14, d_date#12, count#19] -Keys [3]: [_groupingexpression#17, i_item_sk#14, d_date#12] +Input [4]: [_groupingexpression#47, i_item_sk#14, d_date#12, count#19] +Keys [3]: [_groupingexpression#47, i_item_sk#14, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#20] Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#20 AS cnt#22] @@ -400,7 +400,7 @@ Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price (69) Exchange Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] +Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#48] (70) Sort [codegen id : 27] Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] @@ -465,16 +465,16 @@ Output [3]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] (84) ReusedExchange [Reuses operator id: 95] -Output [1]: [d_date_sk#48] +Output [1]: [d_date_sk#49] (85) BroadcastHashJoin [codegen id : 34] Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#48] +Right keys [1]: [d_date_sk#49] Join condition: None (86) Project [codegen id : 34] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#43 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#44 as decimal(12,2)))), DecimalType(18,2), true) AS sales#49] -Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#48] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#43 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#44 as decimal(12,2)))), DecimalType(18,2), true) AS sales#50] +Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#49] (87) Union @@ -482,19 +482,19 @@ Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#48] Input [1]: [sales#40] Keys: [] Functions [1]: [partial_sum(sales#40)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [2]: [sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [2]: [sum#53, isEmpty#54] (89) Exchange -Input [2]: [sum#52, isEmpty#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#54] +Input [2]: [sum#53, isEmpty#54] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#55] (90) HashAggregate [codegen id : 36] -Input [2]: [sum#52, isEmpty#53] +Input [2]: [sum#53, isEmpty#54] Keys: [] Functions [1]: [sum(sales#40)] -Aggregate Attributes [1]: [sum(sales#40)#55] -Results [1]: [sum(sales#40)#55 AS sum(sales)#56] +Aggregate Attributes [1]: [sum(sales#40)#56] +Results [1]: [sum(sales#40)#56 AS sum(sales)#57] ===== Subqueries ===== @@ -507,26 +507,26 @@ BroadcastExchange (95) (91) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#57, d_moy#58] +Output [3]: [d_date_sk#39, d_year#58, d_moy#59] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#57, d_moy#58] +Input [3]: [d_date_sk#39, d_year#58, d_moy#59] (93) Filter [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#57, d_moy#58] -Condition : ((((isnotnull(d_year#57) AND isnotnull(d_moy#58)) AND (d_year#57 = 2000)) AND (d_moy#58 = 2)) AND isnotnull(d_date_sk#39)) +Input [3]: [d_date_sk#39, d_year#58, d_moy#59] +Condition : ((((isnotnull(d_year#58) AND isnotnull(d_moy#59)) AND (d_year#58 = 2000)) AND (d_moy#59 = 2)) AND isnotnull(d_date_sk#39)) (94) Project [codegen id : 1] Output [1]: [d_date_sk#39] -Input [3]: [d_date_sk#39, d_year#57, d_moy#58] +Input [3]: [d_date_sk#39, d_year#58, d_moy#59] (95) BroadcastExchange Input [1]: [d_date_sk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (100) @@ -537,26 +537,26 @@ BroadcastExchange (100) (96) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_date#12, d_year#60] +Output [3]: [d_date_sk#11, d_date#12, d_year#61] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#60] +Input [3]: [d_date_sk#11, d_date#12, d_year#61] (98) Filter [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#60] -Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#11, d_date#12, d_year#61] +Condition : (d_year#61 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) (99) Project [codegen id : 1] Output [2]: [d_date_sk#11, d_date#12] -Input [3]: [d_date_sk#11, d_date#12, d_year#60] +Input [3]: [d_date_sk#11, d_date#12, d_year#61] (100) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#62] Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#37, [id=#38] * HashAggregate (117) @@ -579,89 +579,89 @@ Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] +Output [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#65), dynamicpruningexpression(ss_sold_date_sk#65 IN dynamicpruning#66)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] +Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] (103) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] -Condition : isnotnull(ss_customer_sk#62) +Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] +Condition : isnotnull(ss_customer_sk#63) (104) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#67] +Output [1]: [d_date_sk#68] (105) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#65] -Right keys [1]: [d_date_sk#67] +Left keys [1]: [ss_sold_date_sk#66] +Right keys [1]: [d_date_sk#68] Join condition: None (106) Project [codegen id : 2] -Output [3]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64] -Input [5]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, d_date_sk#67] +Output [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] +Input [5]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66, d_date_sk#68] (107) Exchange -Input [3]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64] -Arguments: hashpartitioning(ss_customer_sk#62, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] +Arguments: hashpartitioning(ss_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] (108) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64] -Arguments: [ss_customer_sk#62 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] +Arguments: [ss_customer_sk#63 ASC NULLS FIRST], false, 0 (109) ReusedExchange [Reuses operator id: 38] -Output [1]: [c_customer_sk#69] +Output [1]: [c_customer_sk#70] (110) Sort [codegen id : 5] -Input [1]: [c_customer_sk#69] -Arguments: [c_customer_sk#69 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#70] +Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 (111) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#62] -Right keys [1]: [c_customer_sk#69] +Left keys [1]: [ss_customer_sk#63] +Right keys [1]: [c_customer_sk#70] Join condition: None (112) Project [codegen id : 6] -Output [3]: [ss_quantity#63, ss_sales_price#64, c_customer_sk#69] -Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, c_customer_sk#69] +Output [3]: [ss_quantity#64, ss_sales_price#65, c_customer_sk#70] +Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, c_customer_sk#70] (113) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#63, ss_sales_price#64, c_customer_sk#69] -Keys [1]: [c_customer_sk#69] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#64 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#70, isEmpty#71] -Results [3]: [c_customer_sk#69, sum#72, isEmpty#73] +Input [3]: [ss_quantity#64, ss_sales_price#65, c_customer_sk#70] +Keys [1]: [c_customer_sk#70] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#71, isEmpty#72] +Results [3]: [c_customer_sk#70, sum#73, isEmpty#74] (114) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#69, sum#72, isEmpty#73] -Keys [1]: [c_customer_sk#69] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#64 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#64 as decimal(12,2)))), DecimalType(18,2), true))#74] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#64 as decimal(12,2)))), DecimalType(18,2), true))#74 AS csales#75] +Input [3]: [c_customer_sk#70, sum#73, isEmpty#74] +Keys [1]: [c_customer_sk#70] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))#75] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))#75 AS csales#76] (115) HashAggregate [codegen id : 6] -Input [1]: [csales#75] +Input [1]: [csales#76] Keys: [] -Functions [1]: [partial_max(csales#75)] -Aggregate Attributes [1]: [max#76] -Results [1]: [max#77] +Functions [1]: [partial_max(csales#76)] +Aggregate Attributes [1]: [max#77] +Results [1]: [max#78] (116) Exchange -Input [1]: [max#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [1]: [max#78] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#79] (117) HashAggregate [codegen id : 7] -Input [1]: [max#77] +Input [1]: [max#78] Keys: [] -Functions [1]: [max(csales#75)] -Aggregate Attributes [1]: [max(csales#75)#79] -Results [1]: [max(csales#75)#79 AS tpcds_cmax#80] +Functions [1]: [max(csales#76)] +Aggregate Attributes [1]: [max(csales#76)#80] +Results [1]: [max(csales#76)#80 AS tpcds_cmax#81] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#65 IN dynamicpruning#66 +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 BroadcastExchange (122) +- * Project (121) +- * Filter (120) @@ -670,26 +670,26 @@ BroadcastExchange (122) (118) Scan parquet default.date_dim -Output [2]: [d_date_sk#67, d_year#81] +Output [2]: [d_date_sk#68, d_year#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#67, d_year#81] +Input [2]: [d_date_sk#68, d_year#82] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#67, d_year#81] -Condition : (d_year#81 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#67)) +Input [2]: [d_date_sk#68, d_year#82] +Condition : (d_year#82 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#68)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#67] -Input [2]: [d_date_sk#67, d_year#81] +Output [1]: [d_date_sk#68] +Input [2]: [d_date_sk#68, d_year#82] (122) BroadcastExchange -Input [1]: [d_date_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#82] +Input [1]: [d_date_sk#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index 9dcc098d3593f..638f5ec3ded62 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -508,19 +508,19 @@ Right keys [1]: [i_item_sk#14] Join condition: None (84) Project [codegen id : 35] -Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#17] +Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#57] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#14, i_item_desc#15] (85) HashAggregate [codegen id : 35] -Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#17] -Keys [3]: [_groupingexpression#17, i_item_sk#14, d_date#12] +Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#57] +Keys [3]: [_groupingexpression#57, i_item_sk#14, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] -Results [4]: [_groupingexpression#17, i_item_sk#14, d_date#12, count#19] +Results [4]: [_groupingexpression#57, i_item_sk#14, d_date#12, count#19] (86) HashAggregate [codegen id : 35] -Input [4]: [_groupingexpression#17, i_item_sk#14, d_date#12, count#19] -Keys [3]: [_groupingexpression#17, i_item_sk#14, d_date#12] +Input [4]: [_groupingexpression#57, i_item_sk#14, d_date#12, count#19] +Keys [3]: [_groupingexpression#57, i_item_sk#14, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#20] Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#20 AS cnt#22] @@ -548,7 +548,7 @@ Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price (92) Exchange Input [4]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] -Arguments: hashpartitioning(ws_bill_customer_sk#52, 5), ENSURE_REQUIREMENTS, [id=#57] +Arguments: hashpartitioning(ws_bill_customer_sk#52, 5), ENSURE_REQUIREMENTS, [id=#58] (93) Sort [codegen id : 37] Input [4]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] @@ -609,23 +609,23 @@ Right keys [1]: [c_customer_sk#29] Join condition: None (106) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#58] +Output [1]: [d_date_sk#59] (107) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#55] -Right keys [1]: [d_date_sk#58] +Right keys [1]: [d_date_sk#59] Join condition: None (108) Project [codegen id : 44] Output [3]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54] -Input [5]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55, d_date_sk#58] +Input [5]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55, d_date_sk#59] (109) ReusedExchange [Reuses operator id: 55] -Output [3]: [c_customer_sk#59, c_first_name#60, c_last_name#61] +Output [3]: [c_customer_sk#60, c_first_name#61, c_last_name#62] (110) Sort [codegen id : 46] -Input [3]: [c_customer_sk#59, c_first_name#60, c_last_name#61] -Arguments: [c_customer_sk#59 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#60, c_first_name#61, c_last_name#62] +Arguments: [c_customer_sk#60 ASC NULLS FIRST], false, 0 (111) ReusedExchange [Reuses operator id: 34] Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] @@ -677,36 +677,36 @@ Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (122) SortMergeJoin [codegen id : 52] -Left keys [1]: [c_customer_sk#59] +Left keys [1]: [c_customer_sk#60] Right keys [1]: [c_customer_sk#29] Join condition: None (123) SortMergeJoin [codegen id : 53] Left keys [1]: [ws_bill_customer_sk#52] -Right keys [1]: [c_customer_sk#59] +Right keys [1]: [c_customer_sk#60] Join condition: None (124) Project [codegen id : 53] -Output [4]: [ws_quantity#53, ws_list_price#54, c_first_name#60, c_last_name#61] -Input [6]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, c_customer_sk#59, c_first_name#60, c_last_name#61] +Output [4]: [ws_quantity#53, ws_list_price#54, c_first_name#61, c_last_name#62] +Input [6]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, c_customer_sk#60, c_first_name#61, c_last_name#62] (125) HashAggregate [codegen id : 53] -Input [4]: [ws_quantity#53, ws_list_price#54, c_first_name#60, c_last_name#61] -Keys [2]: [c_last_name#61, c_first_name#60] +Input [4]: [ws_quantity#53, ws_list_price#54, c_first_name#61, c_last_name#62] +Keys [2]: [c_last_name#62, c_first_name#61] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#62, isEmpty#63] -Results [4]: [c_last_name#61, c_first_name#60, sum#64, isEmpty#65] +Aggregate Attributes [2]: [sum#63, isEmpty#64] +Results [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] (126) Exchange -Input [4]: [c_last_name#61, c_first_name#60, sum#64, isEmpty#65] -Arguments: hashpartitioning(c_last_name#61, c_first_name#60, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] +Arguments: hashpartitioning(c_last_name#62, c_first_name#61, 5), ENSURE_REQUIREMENTS, [id=#67] (127) HashAggregate [codegen id : 54] -Input [4]: [c_last_name#61, c_first_name#60, sum#64, isEmpty#65] -Keys [2]: [c_last_name#61, c_first_name#60] +Input [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] +Keys [2]: [c_last_name#62, c_first_name#61] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#67] -Results [3]: [c_last_name#61, c_first_name#60, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#68] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#68] +Results [3]: [c_last_name#62, c_first_name#61, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#68 AS sales#69] (128) Union @@ -725,26 +725,26 @@ BroadcastExchange (134) (130) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#69, d_moy#70] +Output [3]: [d_date_sk#39, d_year#70, d_moy#71] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (131) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#69, d_moy#70] +Input [3]: [d_date_sk#39, d_year#70, d_moy#71] (132) Filter [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#69, d_moy#70] -Condition : ((((isnotnull(d_year#69) AND isnotnull(d_moy#70)) AND (d_year#69 = 2000)) AND (d_moy#70 = 2)) AND isnotnull(d_date_sk#39)) +Input [3]: [d_date_sk#39, d_year#70, d_moy#71] +Condition : ((((isnotnull(d_year#70) AND isnotnull(d_moy#71)) AND (d_year#70 = 2000)) AND (d_moy#71 = 2)) AND isnotnull(d_date_sk#39)) (133) Project [codegen id : 1] Output [1]: [d_date_sk#39] -Input [3]: [d_date_sk#39, d_year#69, d_moy#70] +Input [3]: [d_date_sk#39, d_year#70, d_moy#71] (134) BroadcastExchange Input [1]: [d_date_sk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (139) @@ -755,26 +755,26 @@ BroadcastExchange (139) (135) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_date#12, d_year#72] +Output [3]: [d_date_sk#11, d_date#12, d_year#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (136) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#72] +Input [3]: [d_date_sk#11, d_date#12, d_year#73] (137) Filter [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#72] -Condition : (d_year#72 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#11, d_date#12, d_year#73] +Condition : (d_year#73 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) (138) Project [codegen id : 1] Output [2]: [d_date_sk#11, d_date#12] -Input [3]: [d_date_sk#11, d_date#12, d_year#72] +Input [3]: [d_date_sk#11, d_date#12, d_year#73] (139) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#74] Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#37, [id=#38] * HashAggregate (156) @@ -797,89 +797,89 @@ Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquer (140) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, ss_sold_date_sk#77] +Output [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (141) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, ss_sold_date_sk#77] +Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] (142) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, ss_sold_date_sk#77] -Condition : isnotnull(ss_customer_sk#74) +Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] +Condition : isnotnull(ss_customer_sk#75) (143) ReusedExchange [Reuses operator id: 161] -Output [1]: [d_date_sk#79] +Output [1]: [d_date_sk#80] (144) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#77] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [ss_sold_date_sk#78] +Right keys [1]: [d_date_sk#80] Join condition: None (145) Project [codegen id : 2] -Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -Input [5]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, ss_sold_date_sk#77, d_date_sk#79] +Output [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] +Input [5]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78, d_date_sk#80] (146) Exchange -Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -Arguments: hashpartitioning(ss_customer_sk#74, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] +Arguments: hashpartitioning(ss_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] (147) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] +Arguments: [ss_customer_sk#75 ASC NULLS FIRST], false, 0 (148) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#81] +Output [1]: [c_customer_sk#82] (149) Sort [codegen id : 5] -Input [1]: [c_customer_sk#81] -Arguments: [c_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#82] +Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 (150) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#74] -Right keys [1]: [c_customer_sk#81] +Left keys [1]: [ss_customer_sk#75] +Right keys [1]: [c_customer_sk#82] Join condition: None (151) Project [codegen id : 6] -Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#81] -Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#81] +Output [3]: [ss_quantity#76, ss_sales_price#77, c_customer_sk#82] +Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, c_customer_sk#82] (152) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#81] -Keys [1]: [c_customer_sk#81] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#82, isEmpty#83] -Results [3]: [c_customer_sk#81, sum#84, isEmpty#85] +Input [3]: [ss_quantity#76, ss_sales_price#77, c_customer_sk#82] +Keys [1]: [c_customer_sk#82] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#83, isEmpty#84] +Results [3]: [c_customer_sk#82, sum#85, isEmpty#86] (153) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#81, sum#84, isEmpty#85] -Keys [1]: [c_customer_sk#81] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#86] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#86 AS csales#87] +Input [3]: [c_customer_sk#82, sum#85, isEmpty#86] +Keys [1]: [c_customer_sk#82] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))#87] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))#87 AS csales#88] (154) HashAggregate [codegen id : 6] -Input [1]: [csales#87] +Input [1]: [csales#88] Keys: [] -Functions [1]: [partial_max(csales#87)] -Aggregate Attributes [1]: [max#88] -Results [1]: [max#89] +Functions [1]: [partial_max(csales#88)] +Aggregate Attributes [1]: [max#89] +Results [1]: [max#90] (155) Exchange -Input [1]: [max#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] +Input [1]: [max#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] (156) HashAggregate [codegen id : 7] -Input [1]: [max#89] +Input [1]: [max#90] Keys: [] -Functions [1]: [max(csales#87)] -Aggregate Attributes [1]: [max(csales#87)#91] -Results [1]: [max(csales#87)#91 AS tpcds_cmax#92] +Functions [1]: [max(csales#88)] +Aggregate Attributes [1]: [max(csales#88)#92] +Results [1]: [max(csales#88)#92 AS tpcds_cmax#93] -Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#78 +Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 BroadcastExchange (161) +- * Project (160) +- * Filter (159) @@ -888,26 +888,26 @@ BroadcastExchange (161) (157) Scan parquet default.date_dim -Output [2]: [d_date_sk#79, d_year#93] +Output [2]: [d_date_sk#80, d_year#94] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (158) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#79, d_year#93] +Input [2]: [d_date_sk#80, d_year#94] (159) Filter [codegen id : 1] -Input [2]: [d_date_sk#79, d_year#93] -Condition : (d_year#93 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#79)) +Input [2]: [d_date_sk#80, d_year#94] +Condition : (d_year#94 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#80)) (160) Project [codegen id : 1] -Output [1]: [d_date_sk#79] -Input [2]: [d_date_sk#79, d_year#93] +Output [1]: [d_date_sk#80] +Input [2]: [d_date_sk#80, d_year#94] (161) BroadcastExchange -Input [1]: [d_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#94] +Input [1]: [d_date_sk#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#95] Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index 474015a8c782b..8931615dbefbd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -362,267 +362,267 @@ Output [11]: [customer_id#25, year_total#26, customer_id#50, customer_first_name Input [12]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#57, customer_id#84, year_total#85] (54) Scan parquet default.customer -Output [8]: [c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58] +Output [8]: [c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58] +Input [8]: [c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94] (56) Filter [codegen id : 14] -Input [8]: [c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58] +Input [8]: [c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94] Condition : (isnotnull(c_customer_sk#87) AND isnotnull(c_customer_id#88)) (57) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#94, cs_ext_discount_amt#73, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] +Output [6]: [cs_bill_customer_sk#95, cs_ext_discount_amt#96, cs_ext_sales_price#97, cs_ext_wholesale_cost#98, cs_ext_list_price#99, cs_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#98), dynamicpruningexpression(cs_sold_date_sk#98 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(cs_sold_date_sk#100), dynamicpruningexpression(cs_sold_date_sk#100 IN dynamicpruning#41)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#94, cs_ext_discount_amt#73, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] +Input [6]: [cs_bill_customer_sk#95, cs_ext_discount_amt#96, cs_ext_sales_price#97, cs_ext_wholesale_cost#98, cs_ext_list_price#99, cs_sold_date_sk#100] (59) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#94, cs_ext_discount_amt#73, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] -Condition : isnotnull(cs_bill_customer_sk#94) +Input [6]: [cs_bill_customer_sk#95, cs_ext_discount_amt#96, cs_ext_sales_price#97, cs_ext_wholesale_cost#98, cs_ext_list_price#99, cs_sold_date_sk#100] +Condition : isnotnull(cs_bill_customer_sk#95) (60) BroadcastExchange -Input [6]: [cs_bill_customer_sk#94, cs_ext_discount_amt#73, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [6]: [cs_bill_customer_sk#95, cs_ext_discount_amt#96, cs_ext_sales_price#97, cs_ext_wholesale_cost#98, cs_ext_list_price#99, cs_sold_date_sk#100] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#101] (61) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_customer_sk#87] -Right keys [1]: [cs_bill_customer_sk#94] +Right keys [1]: [cs_bill_customer_sk#95] Join condition: None (62) Project [codegen id : 14] -Output [12]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, cs_ext_discount_amt#73, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] -Input [14]: [c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, cs_bill_customer_sk#94, cs_ext_discount_amt#73, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] +Output [12]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, cs_ext_discount_amt#96, cs_ext_sales_price#97, cs_ext_wholesale_cost#98, cs_ext_list_price#99, cs_sold_date_sk#100] +Input [14]: [c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, cs_bill_customer_sk#95, cs_ext_discount_amt#96, cs_ext_sales_price#97, cs_ext_wholesale_cost#98, cs_ext_list_price#99, cs_sold_date_sk#100] (63) ReusedExchange [Reuses operator id: 118] -Output [2]: [d_date_sk#100, d_year#101] +Output [2]: [d_date_sk#102, d_year#103] (64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#98] -Right keys [1]: [d_date_sk#100] +Left keys [1]: [cs_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None (65) Project [codegen id : 14] -Output [12]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, cs_ext_discount_amt#73, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, d_year#101] -Input [14]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, cs_ext_discount_amt#73, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98, d_date_sk#100, d_year#101] +Output [12]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, cs_ext_discount_amt#96, cs_ext_sales_price#97, cs_ext_wholesale_cost#98, cs_ext_list_price#99, d_year#103] +Input [14]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, cs_ext_discount_amt#96, cs_ext_sales_price#97, cs_ext_wholesale_cost#98, cs_ext_list_price#99, cs_sold_date_sk#100, d_date_sk#102, d_year#103] (66) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, cs_ext_discount_amt#73, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, d_year#101] -Keys [8]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, d_year#101] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#97 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#96 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#73 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#95 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#102, isEmpty#103] -Results [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, d_year#101, sum#104, isEmpty#105] +Input [12]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, cs_ext_discount_amt#96, cs_ext_sales_price#97, cs_ext_wholesale_cost#98, cs_ext_list_price#99, d_year#103] +Keys [8]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#103] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#99 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#98 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#96 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#97 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#104, isEmpty#105] +Results [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#103, sum#106, isEmpty#107] (67) Exchange -Input [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, d_year#101, sum#104, isEmpty#105] -Arguments: hashpartitioning(c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, d_year#101, 5), ENSURE_REQUIREMENTS, [id=#106] +Input [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#103, sum#106, isEmpty#107] +Arguments: hashpartitioning(c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#103, 5), ENSURE_REQUIREMENTS, [id=#108] (68) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, d_year#101, sum#104, isEmpty#105] -Keys [8]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#58, d_year#101] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#97 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#96 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#73 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#95 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#97 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#96 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#73 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#95 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#81] -Results [2]: [c_customer_id#88 AS customer_id#107, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#97 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#96 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#73 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#95 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#81 AS year_total#108] +Input [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#103, sum#106, isEmpty#107] +Keys [8]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#103] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#99 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#98 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#96 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#97 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#99 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#98 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#96 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#97 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#81] +Results [2]: [c_customer_id#88 AS customer_id#109, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#99 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#98 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#96 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#97 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#81 AS year_total#110] (69) BroadcastExchange -Input [2]: [customer_id#107, year_total#108] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#109] +Input [2]: [customer_id#109, year_total#110] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#111] (70) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#107] -Join condition: (CASE WHEN (year_total#85 > 0.000000) THEN CheckOverflow((promote_precision(year_total#108) / promote_precision(year_total#85)), DecimalType(38,14), true) END > CASE WHEN (year_total#26 > 0.000000) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#26)), DecimalType(38,14), true) END) +Right keys [1]: [customer_id#109] +Join condition: (CASE WHEN (year_total#85 > 0.000000) THEN CheckOverflow((promote_precision(year_total#110) / promote_precision(year_total#85)), DecimalType(38,14), true) END > CASE WHEN (year_total#26 > 0.000000) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#26)), DecimalType(38,14), true) END) (71) Project [codegen id : 24] -Output [10]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#85, year_total#108] -Input [13]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#57, year_total#85, customer_id#107, year_total#108] +Output [10]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#85, year_total#110] +Input [13]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#57, year_total#85, customer_id#109, year_total#110] (72) Scan parquet default.customer -Output [8]: [c_customer_sk#110, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117] +Output [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (73) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#110, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117] +Input [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] (74) Filter [codegen id : 18] -Input [8]: [c_customer_sk#110, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117] -Condition : (isnotnull(c_customer_sk#110) AND isnotnull(c_customer_id#111)) +Input [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] +Condition : (isnotnull(c_customer_sk#112) AND isnotnull(c_customer_id#113)) (75) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#118, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] +Output [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#123), dynamicpruningexpression(ws_sold_date_sk#123 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ws_sold_date_sk#125), dynamicpruningexpression(ws_sold_date_sk#125 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#118, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] +Input [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] (77) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#118, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] -Condition : isnotnull(ws_bill_customer_sk#118) +Input [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Condition : isnotnull(ws_bill_customer_sk#120) (78) BroadcastExchange -Input [6]: [ws_bill_customer_sk#118, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#124] +Input [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#126] (79) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#110] -Right keys [1]: [ws_bill_customer_sk#118] +Left keys [1]: [c_customer_sk#112] +Right keys [1]: [ws_bill_customer_sk#120] Join condition: None (80) Project [codegen id : 18] -Output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] -Input [14]: [c_customer_sk#110, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, ws_bill_customer_sk#118, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] +Output [12]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Input [14]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] (81) ReusedExchange [Reuses operator id: 114] -Output [2]: [d_date_sk#125, d_year#126] +Output [2]: [d_date_sk#127, d_year#128] (82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#123] -Right keys [1]: [d_date_sk#125] +Left keys [1]: [ws_sold_date_sk#125] +Right keys [1]: [d_date_sk#127] Join condition: None (83) Project [codegen id : 18] -Output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#126] -Input [14]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123, d_date_sk#125, d_year#126] +Output [12]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, d_year#128] +Input [14]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125, d_date_sk#127, d_year#128] (84) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#126] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#122 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#121 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#119 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#120 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#127, isEmpty#128] -Results [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126, sum#129, isEmpty#130] +Input [12]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, d_year#128] +Keys [8]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#124 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#123 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#121 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#122 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#129, isEmpty#130] +Results [10]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, sum#131, isEmpty#132] (85) Exchange -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126, sum#129, isEmpty#130] -Arguments: hashpartitioning(c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126, 5), ENSURE_REQUIREMENTS, [id=#131] +Input [10]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, sum#131, isEmpty#132] +Arguments: hashpartitioning(c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, 5), ENSURE_REQUIREMENTS, [id=#133] (86) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126, sum#129, isEmpty#130] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#122 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#121 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#119 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#120 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#122 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#121 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#119 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#120 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#132] -Results [2]: [c_customer_id#111 AS customer_id#133, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#122 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#121 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#119 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#120 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#132 AS year_total#134] +Input [10]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, sum#131, isEmpty#132] +Keys [8]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#124 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#123 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#121 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#122 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#124 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#123 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#121 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#122 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#134] +Results [2]: [c_customer_id#113 AS customer_id#135, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#124 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#123 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#121 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#122 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#134 AS year_total#136] (87) Filter [codegen id : 19] -Input [2]: [customer_id#133, year_total#134] -Condition : (isnotnull(year_total#134) AND (year_total#134 > 0.000000)) +Input [2]: [customer_id#135, year_total#136] +Condition : (isnotnull(year_total#136) AND (year_total#136 > 0.000000)) (88) Project [codegen id : 19] -Output [2]: [customer_id#133 AS customer_id#135, year_total#134 AS year_total#136] -Input [2]: [customer_id#133, year_total#134] +Output [2]: [customer_id#135 AS customer_id#137, year_total#136 AS year_total#138] +Input [2]: [customer_id#135, year_total#136] (89) BroadcastExchange -Input [2]: [customer_id#135, year_total#136] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#137] +Input [2]: [customer_id#137, year_total#138] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#139] (90) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#135] +Right keys [1]: [customer_id#137] Join condition: None (91) Project [codegen id : 24] -Output [11]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#85, year_total#108, year_total#136] -Input [12]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#85, year_total#108, customer_id#135, year_total#136] +Output [11]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#85, year_total#110, year_total#138] +Input [12]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#85, year_total#110, customer_id#137, year_total#138] (92) Scan parquet default.customer -Output [8]: [c_customer_sk#138, c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145] +Output [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (93) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#138, c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145] +Input [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] (94) Filter [codegen id : 22] -Input [8]: [c_customer_sk#138, c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145] -Condition : (isnotnull(c_customer_sk#138) AND isnotnull(c_customer_id#139)) +Input [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] +Condition : (isnotnull(c_customer_sk#140) AND isnotnull(c_customer_id#141)) (95) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Output [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_sold_date_sk#153 IN dynamicpruning#41)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] (97) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] -Condition : isnotnull(ws_bill_customer_sk#146) +Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] +Condition : isnotnull(ws_bill_customer_sk#148) (98) BroadcastExchange -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#152] +Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] (99) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#138] -Right keys [1]: [ws_bill_customer_sk#146] +Left keys [1]: [c_customer_sk#140] +Right keys [1]: [ws_bill_customer_sk#148] Join condition: None (100) Project [codegen id : 22] -Output [12]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] -Input [14]: [c_customer_sk#138, c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Output [12]: [c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] +Input [14]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] (101) ReusedExchange [Reuses operator id: 118] -Output [2]: [d_date_sk#153, d_year#154] +Output [2]: [d_date_sk#155, d_year#156] (102) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#151] -Right keys [1]: [d_date_sk#153] +Left keys [1]: [ws_sold_date_sk#153] +Right keys [1]: [d_date_sk#155] Join condition: None (103) Project [codegen id : 22] -Output [12]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#154] -Input [14]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151, d_date_sk#153, d_year#154] +Output [12]: [c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, d_year#156] +Input [14]: [c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153, d_date_sk#155, d_year#156] (104) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#154] -Keys [8]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#155, isEmpty#156] -Results [10]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154, sum#157, isEmpty#158] +Input [12]: [c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, d_year#156] +Keys [8]: [c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, d_year#156] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#152 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#151 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#149 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#150 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#157, isEmpty#158] +Results [10]: [c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, d_year#156, sum#159, isEmpty#160] (105) Exchange -Input [10]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154, sum#157, isEmpty#158] -Arguments: hashpartitioning(c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154, 5), ENSURE_REQUIREMENTS, [id=#159] +Input [10]: [c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, d_year#156, sum#159, isEmpty#160] +Arguments: hashpartitioning(c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, d_year#156, 5), ENSURE_REQUIREMENTS, [id=#161] (106) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154, sum#157, isEmpty#158] -Keys [8]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#132] -Results [2]: [c_customer_id#139 AS customer_id#160, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#132 AS year_total#161] +Input [10]: [c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, d_year#156, sum#159, isEmpty#160] +Keys [8]: [c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147, d_year#156] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#152 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#151 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#149 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#150 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#152 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#151 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#149 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#150 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#134] +Results [2]: [c_customer_id#141 AS customer_id#162, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#152 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#151 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#149 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#150 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#134 AS year_total#163] (107) BroadcastExchange -Input [2]: [customer_id#160, year_total#161] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#162] +Input [2]: [customer_id#162, year_total#163] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#164] (108) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#160] -Join condition: (CASE WHEN (year_total#85 > 0.000000) THEN CheckOverflow((promote_precision(year_total#108) / promote_precision(year_total#85)), DecimalType(38,14), true) END > CASE WHEN (year_total#136 > 0.000000) THEN CheckOverflow((promote_precision(year_total#161) / promote_precision(year_total#136)), DecimalType(38,14), true) END) +Right keys [1]: [customer_id#162] +Join condition: (CASE WHEN (year_total#85 > 0.000000) THEN CheckOverflow((promote_precision(year_total#110) / promote_precision(year_total#85)), DecimalType(38,14), true) END > CASE WHEN (year_total#138 > 0.000000) THEN CheckOverflow((promote_precision(year_total#163) / promote_precision(year_total#138)), DecimalType(38,14), true) END) (109) Project [codegen id : 24] Output [7]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56] -Input [13]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#85, year_total#108, year_total#136, customer_id#160, year_total#161] +Input [13]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#85, year_total#110, year_total#138, customer_id#162, year_total#163] (110) TakeOrderedAndProject Input [7]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56] @@ -653,7 +653,7 @@ Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_ (114) BroadcastExchange Input [2]: [d_date_sk#17, d_year#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#163] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#165] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#40 IN dynamicpruning#41 BroadcastExchange (118) @@ -678,14 +678,14 @@ Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_ (118) BroadcastExchange Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#164] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#166] Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#98 IN dynamicpruning#41 +Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#100 IN dynamicpruning#41 -Subquery:5 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#123 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#125 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#41 +Subquery:6 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#153 IN dynamicpruning#41 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index 831fb512ea601..693a853440d32 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -188,233 +188,233 @@ Input [4]: [cs_catalog_page_sk#44, cs_ext_sales_price#45, cs_net_profit#46, cs_s Condition : isnotnull(cs_catalog_page_sk#44) (25) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#44 AS page_sk#25, cs_sold_date_sk#47 AS date_sk#48, cs_ext_sales_price#45 AS sales_price#49, cs_net_profit#46 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] +Output [6]: [cs_catalog_page_sk#44 AS page_sk#48, cs_sold_date_sk#47 AS date_sk#49, cs_ext_sales_price#45 AS sales_price#50, cs_net_profit#46 AS profit#51, 0.00 AS return_amt#52, 0.00 AS net_loss#53] Input [4]: [cs_catalog_page_sk#44, cs_ext_sales_price#45, cs_net_profit#46, cs_sold_date_sk#47] (26) Scan parquet default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] (28) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Condition : isnotnull(cr_catalog_page_sk#54) (29) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Output [6]: [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] (30) Union (31) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#64] (32) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [date_sk#49] +Right keys [1]: [d_date_sk#64] Join condition: None (33) Project [codegen id : 11] -Output [5]: [page_sk#25, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#25, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] +Output [5]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53] +Input [7]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53, d_date_sk#64] (34) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] (36) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Condition : isnotnull(cp_catalog_page_sk#65) (37) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#67] (38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#25] -Right keys [1]: [cp_catalog_page_sk#64] +Left keys [1]: [page_sk#48] +Right keys [1]: [cp_catalog_page_sk#65] Join condition: None (39) Project [codegen id : 11] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] -Input [7]: [page_sk#25, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] +Output [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] +Input [7]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_sk#65, cp_catalog_page_id#66] (40) HashAggregate [codegen id : 11] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] -Keys [1]: [cp_catalog_page_id#65] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] -Results [5]: [cp_catalog_page_id#65, sum#71, sum#72, sum#73, sum#74] +Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [partial_sum(UnscaledValue(sales_price#50)), partial_sum(UnscaledValue(return_amt#52)), partial_sum(UnscaledValue(profit#51)), partial_sum(UnscaledValue(net_loss#53))] +Aggregate Attributes [4]: [sum#68, sum#69, sum#70, sum#71] +Results [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] (41) Exchange -Input [5]: [cp_catalog_page_id#65, sum#71, sum#72, sum#73, sum#74] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, [id=#76] (42) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#65, sum#71, sum#72, sum#73, sum#74] -Keys [1]: [cp_catalog_page_id#65] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#76, sum(UnscaledValue(return_amt#51))#77, sum(UnscaledValue(profit#50))#78, sum(UnscaledValue(net_loss#52))#79] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#49))#76,17,2) AS sales#80, MakeDecimal(sum(UnscaledValue(return_amt#51))#77,17,2) AS returns#81, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#50))#78,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#52))#79,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#82, catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#65) AS id#84] +Input [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [sum(UnscaledValue(sales_price#50)), sum(UnscaledValue(return_amt#52)), sum(UnscaledValue(profit#51)), sum(UnscaledValue(net_loss#53))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#50))#77, sum(UnscaledValue(return_amt#52))#78, sum(UnscaledValue(profit#51))#79, sum(UnscaledValue(net_loss#53))#80] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#50))#77,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#52))#78,17,2) AS returns#82, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#51))#79,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#53))#80,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#66) AS id#85] (43) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] +Output [4]: [ws_web_site_sk#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#88), dynamicpruningexpression(ws_sold_date_sk#88 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] +Input [4]: [ws_web_site_sk#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] (45) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] -Condition : isnotnull(ws_web_site_sk#85) +Input [4]: [ws_web_site_sk#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Condition : isnotnull(ws_web_site_sk#86) (46) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#85 AS wsr_web_site_sk#89, ws_sold_date_sk#88 AS date_sk#90, ws_ext_sales_price#86 AS sales_price#91, ws_net_profit#87 AS profit#92, 0.00 AS return_amt#93, 0.00 AS net_loss#94] -Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] +Output [6]: [ws_web_site_sk#86 AS wsr_web_site_sk#90, ws_sold_date_sk#89 AS date_sk#91, ws_ext_sales_price#87 AS sales_price#92, ws_net_profit#88 AS profit#93, 0.00 AS return_amt#94, 0.00 AS net_loss#95] +Input [4]: [ws_web_site_sk#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] (47) Scan parquet default.web_returns -Output [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] +Output [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#99), dynamicpruningexpression(wr_returned_date_sk#99 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#100), dynamicpruningexpression(wr_returned_date_sk#100 IN dynamicpruning#5)] ReadSchema: struct (48) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] +Input [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] (49) BroadcastExchange -Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [id=#100] +Input [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [id=#101] (50) Scan parquet default.web_sales -Output [4]: [ws_item_sk#34, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Output [4]: [ws_item_sk#102, ws_web_site_sk#103, ws_order_number#104, ws_sold_date_sk#105] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (51) ColumnarToRow -Input [4]: [ws_item_sk#34, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Input [4]: [ws_item_sk#102, ws_web_site_sk#103, ws_order_number#104, ws_sold_date_sk#105] (52) Filter -Input [4]: [ws_item_sk#34, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] -Condition : ((isnotnull(ws_item_sk#34) AND isnotnull(ws_order_number#102)) AND isnotnull(ws_web_site_sk#101)) +Input [4]: [ws_item_sk#102, ws_web_site_sk#103, ws_order_number#104, ws_sold_date_sk#105] +Condition : ((isnotnull(ws_item_sk#102) AND isnotnull(ws_order_number#104)) AND isnotnull(ws_web_site_sk#103)) (53) Project -Output [3]: [ws_item_sk#34, ws_web_site_sk#101, ws_order_number#102] -Input [4]: [ws_item_sk#34, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Output [3]: [ws_item_sk#102, ws_web_site_sk#103, ws_order_number#104] +Input [4]: [ws_item_sk#102, ws_web_site_sk#103, ws_order_number#104, ws_sold_date_sk#105] (54) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#95, wr_order_number#96] -Right keys [2]: [ws_item_sk#34, ws_order_number#102] +Left keys [2]: [wr_item_sk#96, wr_order_number#97] +Right keys [2]: [ws_item_sk#102, ws_order_number#104] Join condition: None (55) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#101 AS wsr_web_site_sk#104, wr_returned_date_sk#99 AS date_sk#105, 0.00 AS sales_price#106, 0.00 AS profit#107, wr_return_amt#97 AS return_amt#108, wr_net_loss#98 AS net_loss#109] -Input [8]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99, ws_item_sk#34, ws_web_site_sk#101, ws_order_number#102] +Output [6]: [ws_web_site_sk#103 AS wsr_web_site_sk#106, wr_returned_date_sk#100 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#98 AS return_amt#110, wr_net_loss#99 AS net_loss#111] +Input [8]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100, ws_item_sk#102, ws_web_site_sk#103, ws_order_number#104] (56) Union (57) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#110] +Output [1]: [d_date_sk#112] (58) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#90] -Right keys [1]: [d_date_sk#110] +Left keys [1]: [date_sk#91] +Right keys [1]: [d_date_sk#112] Join condition: None (59) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94] -Input [7]: [wsr_web_site_sk#89, date_sk#90, sales_price#91, profit#92, return_amt#93, net_loss#94, d_date_sk#110] +Output [5]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95] +Input [7]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, d_date_sk#112] (60) Scan parquet default.web_site -Output [2]: [web_site_sk#111, web_site_id#112] +Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#111, web_site_id#112] +Input [2]: [web_site_sk#113, web_site_id#114] (62) Filter [codegen id : 17] -Input [2]: [web_site_sk#111, web_site_id#112] -Condition : isnotnull(web_site_sk#111) +Input [2]: [web_site_sk#113, web_site_id#114] +Condition : isnotnull(web_site_sk#113) (63) BroadcastExchange -Input [2]: [web_site_sk#111, web_site_id#112] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#113] +Input [2]: [web_site_sk#113, web_site_id#114] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] (64) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#89] -Right keys [1]: [web_site_sk#111] +Left keys [1]: [wsr_web_site_sk#90] +Right keys [1]: [web_site_sk#113] Join condition: None (65) Project [codegen id : 18] -Output [5]: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] -Input [7]: [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_sk#111, web_site_id#112] +Output [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#114] +Input [7]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_sk#113, web_site_id#114] (66) HashAggregate [codegen id : 18] -Input [5]: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] -Keys [1]: [web_site_id#112] -Functions [4]: [partial_sum(UnscaledValue(sales_price#91)), partial_sum(UnscaledValue(return_amt#93)), partial_sum(UnscaledValue(profit#92)), partial_sum(UnscaledValue(net_loss#94))] -Aggregate Attributes [4]: [sum#114, sum#115, sum#116, sum#117] -Results [5]: [web_site_id#112, sum#118, sum#119, sum#120, sum#121] +Input [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#114] +Keys [1]: [web_site_id#114] +Functions [4]: [partial_sum(UnscaledValue(sales_price#92)), partial_sum(UnscaledValue(return_amt#94)), partial_sum(UnscaledValue(profit#93)), partial_sum(UnscaledValue(net_loss#95))] +Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] +Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] (67) Exchange -Input [5]: [web_site_id#112, sum#118, sum#119, sum#120, sum#121] -Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, [id=#122] +Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] (68) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#112, sum#118, sum#119, sum#120, sum#121] -Keys [1]: [web_site_id#112] -Functions [4]: [sum(UnscaledValue(sales_price#91)), sum(UnscaledValue(return_amt#93)), sum(UnscaledValue(profit#92)), sum(UnscaledValue(net_loss#94))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#91))#123, sum(UnscaledValue(return_amt#93))#124, sum(UnscaledValue(profit#92))#125, sum(UnscaledValue(net_loss#94))#126] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#91))#123,17,2) AS sales#127, MakeDecimal(sum(UnscaledValue(return_amt#93))#124,17,2) AS returns#128, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#92))#125,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#94))#126,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#129, web channel AS channel#130, concat(web_site, web_site_id#112) AS id#131] +Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Keys [1]: [web_site_id#114] +Functions [4]: [sum(UnscaledValue(sales_price#92)), sum(UnscaledValue(return_amt#94)), sum(UnscaledValue(profit#93)), sum(UnscaledValue(net_loss#95))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#92))#125, sum(UnscaledValue(return_amt#94))#126, sum(UnscaledValue(profit#93))#127, sum(UnscaledValue(net_loss#95))#128] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#92))#125,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#94))#126,17,2) AS returns#130, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#93))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#95))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#131, web channel AS channel#132, concat(web_site, web_site_id#114) AS id#133] (69) Union (70) Expand [codegen id : 20] Input [5]: [sales#39, returns#40, profit#41, channel#42, id#43] -Arguments: [[sales#39, returns#40, profit#41, channel#42, id#43, 0], [sales#39, returns#40, profit#41, channel#42, null, 1], [sales#39, returns#40, profit#41, null, null, 3]], [sales#39, returns#40, profit#41, channel#132, id#133, spark_grouping_id#134] +Arguments: [[sales#39, returns#40, profit#41, channel#42, id#43, 0], [sales#39, returns#40, profit#41, channel#42, null, 1], [sales#39, returns#40, profit#41, null, null, 3]], [sales#39, returns#40, profit#41, channel#134, id#135, spark_grouping_id#136] (71) HashAggregate [codegen id : 20] -Input [6]: [sales#39, returns#40, profit#41, channel#132, id#133, spark_grouping_id#134] -Keys [3]: [channel#132, id#133, spark_grouping_id#134] +Input [6]: [sales#39, returns#40, profit#41, channel#134, id#135, spark_grouping_id#136] +Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] -Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Results [9]: [channel#132, id#133, spark_grouping_id#134, sum#141, isEmpty#75, sum#142, isEmpty#143, sum#144, isEmpty#145] +Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Results [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] (72) Exchange -Input [9]: [channel#132, id#133, spark_grouping_id#134, sum#141, isEmpty#75, sum#142, isEmpty#143, sum#144, isEmpty#145] -Arguments: hashpartitioning(channel#132, id#133, spark_grouping_id#134, 5), ENSURE_REQUIREMENTS, [id=#146] +Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Arguments: hashpartitioning(channel#134, id#135, spark_grouping_id#136, 5), ENSURE_REQUIREMENTS, [id=#149] (73) HashAggregate [codegen id : 21] -Input [9]: [channel#132, id#133, spark_grouping_id#134, sum#141, isEmpty#75, sum#142, isEmpty#143, sum#144, isEmpty#145] -Keys [3]: [channel#132, id#133, spark_grouping_id#134] +Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#147, sum(returns#40)#148, sum(profit#41)#149] -Results [5]: [channel#132, id#133, sum(sales#39)#147 AS sales#150, sum(returns#40)#148 AS returns#151, sum(profit#41)#149 AS profit#152] +Aggregate Attributes [3]: [sum(sales#39)#150, sum(returns#40)#151, sum(profit#41)#152] +Results [5]: [channel#134, id#135, sum(sales#39)#150 AS sales#153, sum(returns#40)#151 AS returns#154, sum(profit#41)#152 AS profit#155] (74) TakeOrderedAndProject -Input [5]: [channel#132, id#133, sales#150, returns#151, profit#152] -Arguments: 100, [channel#132 ASC NULLS FIRST, id#133 ASC NULLS FIRST], [channel#132, id#133, sales#150, returns#151, profit#152] +Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155] +Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] ===== Subqueries ===== @@ -427,35 +427,35 @@ BroadcastExchange (79) (75) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_date#153] +Output [2]: [d_date_sk#22, d_date#156] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#153] +Input [2]: [d_date_sk#22, d_date#156] (77) Filter [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#153] -Condition : (((isnotnull(d_date#153) AND (d_date#153 >= 2000-08-23)) AND (d_date#153 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#22, d_date#156] +Condition : (((isnotnull(d_date#156) AND (d_date#156 >= 2000-08-23)) AND (d_date#156 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) (78) Project [codegen id : 1] Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#153] +Input [2]: [d_date_sk#22, d_date#156] (79) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#154] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#157] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#88 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#99 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#100 IN dynamicpruning#5 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala index b69aaaf439603..7ee533ac26d2b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.expressions.{And, GreaterThan, LessThan, Literal, Or} -import org.apache.spark.sql.catalyst.plans.logical.{Filter, Project, RepartitionOperation, WithCTE} +import org.apache.spark.sql.catalyst.plans.logical.WithCTE import org.apache.spark.sql.execution.adaptive._ import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.internal.SQLConf @@ -43,7 +42,7 @@ abstract class CTEInlineSuiteBase """.stripMargin) checkAnswer(df, Nil) assert( - df.queryExecution.optimizedPlan.find(_.isInstanceOf[RepartitionOperation]).nonEmpty, + df.queryExecution.optimizedPlan.find(_.isInstanceOf[WithCTE]).nonEmpty, "Non-deterministic With-CTE with multiple references should be not inlined.") } } @@ -60,7 +59,7 @@ abstract class CTEInlineSuiteBase """.stripMargin) checkAnswer(df, Nil) assert( - df.queryExecution.optimizedPlan.find(_.isInstanceOf[RepartitionOperation]).nonEmpty, + df.queryExecution.optimizedPlan.find(_.isInstanceOf[WithCTE]).nonEmpty, "Non-deterministic With-CTE with multiple references should be not inlined.") } } @@ -80,7 +79,7 @@ abstract class CTEInlineSuiteBase df.queryExecution.analyzed.find(_.isInstanceOf[WithCTE]).nonEmpty, "With-CTE should not be inlined in analyzed plan.") assert( - df.queryExecution.optimizedPlan.find(_.isInstanceOf[RepartitionOperation]).isEmpty, + df.queryExecution.optimizedPlan.find(_.isInstanceOf[WithCTE]).isEmpty, "With-CTE with one reference should be inlined in optimized plan.") } } @@ -108,8 +107,8 @@ abstract class CTEInlineSuiteBase "With-CTE should contain 2 CTE defs after analysis.") assert( df.queryExecution.optimizedPlan.collect { - case r: RepartitionOperation => r - }.length == 6, + case WithCTE(_, cteDefs) => cteDefs + }.head.length == 2, "With-CTE should contain 2 CTE def after optimization.") } } @@ -137,8 +136,8 @@ abstract class CTEInlineSuiteBase "With-CTE should contain 2 CTE defs after analysis.") assert( df.queryExecution.optimizedPlan.collect { - case r: RepartitionOperation => r - }.length == 4, + case WithCTE(_, cteDefs) => cteDefs + }.head.length == 1, "One CTE def should be inlined after optimization.") } } @@ -164,7 +163,7 @@ abstract class CTEInlineSuiteBase "With-CTE should contain 2 CTE defs after analysis.") assert( df.queryExecution.optimizedPlan.collect { - case r: RepartitionOperation => r + case WithCTE(_, cteDefs) => cteDefs }.isEmpty, "CTEs with one reference should all be inlined after optimization.") } @@ -249,7 +248,7 @@ abstract class CTEInlineSuiteBase "With-CTE should contain 2 CTE defs after analysis.") assert( df.queryExecution.optimizedPlan.collect { - case r: RepartitionOperation => r + case WithCTE(_, cteDefs) => cteDefs }.isEmpty, "Deterministic CTEs should all be inlined after optimization.") } @@ -273,214 +272,6 @@ abstract class CTEInlineSuiteBase assert(ex.message.contains("Table or view not found: v1")) } } - - test("CTE Predicate push-down and column pruning") { - withView("t") { - Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") - val df = sql( - s"""with - |v as ( - | select c1, c2, 's' c3, rand() c4 from t - |), - |vv as ( - | select v1.c1, v1.c2, rand() c5 from v v1, v v2 - | where v1.c1 > 0 and v1.c3 = 's' and v1.c2 = v2.c2 - |) - |select vv1.c1, vv1.c2, vv2.c1, vv2.c2 from vv vv1, vv vv2 - |where vv1.c2 > 0 and vv2.c2 > 0 and vv1.c1 = vv2.c1 - """.stripMargin) - checkAnswer(df, Row(1, 2, 1, 2) :: Nil) - assert( - df.queryExecution.analyzed.collect { - case WithCTE(_, cteDefs) => cteDefs - }.head.length == 2, - "With-CTE should contain 2 CTE defs after analysis.") - val cteRepartitions = df.queryExecution.optimizedPlan.collect { - case r: RepartitionOperation => r - } - assert(cteRepartitions.length == 6, - "CTE should not be inlined after optimization.") - val distinctCteRepartitions = cteRepartitions.map(_.canonicalized).distinct - // Check column pruning and predicate push-down. - assert(distinctCteRepartitions.length == 2) - assert(distinctCteRepartitions(1).collectFirst { - case p: Project if p.projectList.length == 3 => p - }.isDefined, "CTE columns should be pruned.") - assert(distinctCteRepartitions(1).collectFirst { - case f: Filter if f.condition.semanticEquals(GreaterThan(f.output(1), Literal(0))) => f - }.isDefined, "Predicate 'c2 > 0' should be pushed down to the CTE def 'v'.") - assert(distinctCteRepartitions(0).collectFirst { - case f: Filter if f.condition.find(_.semanticEquals(f.output(0))).isDefined => f - }.isDefined, "CTE 'vv' definition contains predicate 'c1 > 0'.") - assert(distinctCteRepartitions(1).collectFirst { - case f: Filter if f.condition.find(_.semanticEquals(f.output(0))).isDefined => f - }.isEmpty, "Predicate 'c1 > 0' should be not pushed down to the CTE def 'v'.") - // Check runtime repartition reuse. - assert( - collectWithSubqueries(df.queryExecution.executedPlan) { - case r: ReusedExchangeExec => r - }.length == 2, - "CTE repartition is reused.") - } - } - - test("CTE Predicate push-down and column pruning - combined predicate") { - withView("t") { - Seq((0, 1, 2), (1, 2, 3)).toDF("c1", "c2", "c3").createOrReplaceTempView("t") - val df = sql( - s"""with - |v as ( - | select c1, c2, c3, rand() c4 from t - |), - |vv as ( - | select v1.c1, v1.c2, rand() c5 from v v1, v v2 - | where v1.c1 > 0 and v2.c3 < 5 and v1.c2 = v2.c2 - |) - |select vv1.c1, vv1.c2, vv2.c1, vv2.c2 from vv vv1, vv vv2 - |where vv1.c2 > 0 and vv2.c2 > 0 and vv1.c1 = vv2.c1 - """.stripMargin) - checkAnswer(df, Row(1, 2, 1, 2) :: Nil) - assert( - df.queryExecution.analyzed.collect { - case WithCTE(_, cteDefs) => cteDefs - }.head.length == 2, - "With-CTE should contain 2 CTE defs after analysis.") - val cteRepartitions = df.queryExecution.optimizedPlan.collect { - case r: RepartitionOperation => r - } - assert(cteRepartitions.length == 6, - "CTE should not be inlined after optimization.") - val distinctCteRepartitions = cteRepartitions.map(_.canonicalized).distinct - // Check column pruning and predicate push-down. - assert(distinctCteRepartitions.length == 2) - assert(distinctCteRepartitions(1).collectFirst { - case p: Project if p.projectList.length == 3 => p - }.isDefined, "CTE columns should be pruned.") - assert( - distinctCteRepartitions(1).collectFirst { - case f: Filter - if f.condition.semanticEquals( - And( - GreaterThan(f.output(1), Literal(0)), - Or( - GreaterThan(f.output(0), Literal(0)), - LessThan(f.output(2), Literal(5))))) => - f - }.isDefined, - "Predicate 'c2 > 0 AND (c1 > 0 OR c3 < 5)' should be pushed down to the CTE def 'v'.") - // Check runtime repartition reuse. - assert( - collectWithSubqueries(df.queryExecution.executedPlan) { - case r: ReusedExchangeExec => r - }.length == 2, - "CTE repartition is reused.") - } - } - - test("Views with CTEs - 1 temp view") { - withView("t", "t2") { - Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") - sql( - s"""with - |v as ( - | select c1 + c2 c3 from t - |) - |select sum(c3) s from v - """.stripMargin).createOrReplaceTempView("t2") - val df = sql( - s"""with - |v as ( - | select c1 * c2 c3 from t - |) - |select sum(c3) from v except select s from t2 - """.stripMargin) - checkAnswer(df, Row(2) :: Nil) - } - } - - test("Views with CTEs - 2 temp views") { - withView("t", "t2", "t3") { - Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") - sql( - s"""with - |v as ( - | select c1 + c2 c3 from t - |) - |select sum(c3) s from v - """.stripMargin).createOrReplaceTempView("t2") - sql( - s"""with - |v as ( - | select c1 * c2 c3 from t - |) - |select sum(c3) s from v - """.stripMargin).createOrReplaceTempView("t3") - val df = sql("select s from t3 except select s from t2") - checkAnswer(df, Row(2) :: Nil) - } - } - - test("Views with CTEs - temp view + sql view") { - withTable("t") { - withView ("t2", "t3") { - Seq((0, 1), (1, 2)).toDF("c1", "c2").write.saveAsTable("t") - sql( - s"""with - |v as ( - | select c1 + c2 c3 from t - |) - |select sum(c3) s from v - """.stripMargin).createOrReplaceTempView("t2") - sql( - s"""create view t3 as - |with - |v as ( - | select c1 * c2 c3 from t - |) - |select sum(c3) s from v - """.stripMargin) - val df = sql("select s from t3 except select s from t2") - checkAnswer(df, Row(2) :: Nil) - } - } - } - - test("Union of Dataframes with CTEs") { - val a = spark.sql("with t as (select 1 as n) select * from t ") - val b = spark.sql("with t as (select 2 as n) select * from t ") - val df = a.union(b) - checkAnswer(df, Row(1) :: Row(2) :: Nil) - } - - test("CTE definitions out of original order when not inlined") { - withView("t1", "t2") { - Seq((1, 2, 10, 100), (2, 3, 20, 200)).toDF("workspace_id", "issue_id", "shard_id", "field_id") - .createOrReplaceTempView("issue_current") - withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> - "org.apache.spark.sql.catalyst.optimizer.InlineCTE") { - val df = sql( - """ - |WITH cte_0 AS ( - | SELECT workspace_id, issue_id, shard_id, field_id FROM issue_current - |), - |cte_1 AS ( - | WITH filtered_source_table AS ( - | SELECT * FROM cte_0 WHERE shard_id in ( 10 ) - | ) - | SELECT source_table.workspace_id, field_id FROM cte_0 source_table - | INNER JOIN ( - | SELECT workspace_id, issue_id FROM filtered_source_table GROUP BY 1, 2 - | ) target_table - | ON source_table.issue_id = target_table.issue_id - | AND source_table.workspace_id = target_table.workspace_id - | WHERE source_table.shard_id IN ( 10 ) - |) - |SELECT * FROM cte_1 - """.stripMargin) - checkAnswer(df, Row(1, 100) :: Nil) - } - } - } } class CTEInlineSuiteAEOff extends CTEInlineSuiteBase with DisableAdaptiveExecutionSuite diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 581108b81b6a4..1515abb052a7f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -579,21 +579,6 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark |select * from q1 union all select * from q2""".stripMargin), Row(5, "5") :: Row(4, "4") :: Nil) - // inner CTE relation refers to outer CTE relation. - withSQLConf(SQLConf.LEGACY_CTE_PRECEDENCE_POLICY.key -> "CORRECTED") { - checkAnswer( - sql( - """ - |with temp1 as (select 1 col), - |temp2 as ( - | with temp1 as (select col + 1 AS col from temp1), - | temp3 as (select col + 1 from temp1) - | select * from temp3 - |) - |select * from temp2 - |""".stripMargin), - Row(3)) - } } test("Allow only a single WITH clause per query") {