diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala index cf052f865ea90..ba958b3db031b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala @@ -56,6 +56,9 @@ case class BloomFilterMightContain( case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) => TypeCheckResult.TypeCheckSuccess + case GetStructField(subquery: PlanExpression[_], _, _) + if !subquery.containsPattern(OUTER_REFERENCE) => + TypeCheckResult.TypeCheckSuccess case _ => TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " + "should be either a constant value or a scalar subquery expression") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala new file mode 100644 index 0000000000000..44f3b653de75c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala @@ -0,0 +1,389 @@ +/* + * 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.ArrayBuffer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CTERelationDef, CTERelationRef, Filter, Join, LogicalPlan, Project, Subquery, WithCTE} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.DataType + +/** + * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar + * values once. + * + * The process is the following: + * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache + * of already seen subquery plans. If merge is possible then cache is updated with the merged + * subquery plan, if not then the new subquery plan is added to the cache. + * During this first traversal each [[ScalarSubquery]] expression is replaced to a temporal + * [[ScalarSubqueryReference]] reference pointing to its cached version. + * The cache uses a flag to keep track of if a cache entry is a result of merging 2 or more + * plans, or it is a plan that was seen only once. + * Merged plans in the cache get a "Header", that contains the list of attributes form the scalar + * return value of a merged subquery. + * - A second traversal checks if there are merged subqueries in the cache and builds a `WithCTE` + * node from these queries. The `CTERelationDef` nodes contain the merged subquery in the + * following form: + * `Project(Seq(CreateNamedStruct(name1, attribute1, ...) AS mergedValue), mergedSubqueryPlan)` + * and the definitions are flagged that they host a subquery, that can return maximum one row. + * During the second traversal [[ScalarSubqueryReference]] expressions that pont to a merged + * subquery is either transformed to a `GetStructField(ScalarSubquery(CTERelationRef(...)))` + * expression or restored to the original [[ScalarSubquery]]. + * + * Eg. the following query: + * + * SELECT + * (SELECT avg(a) FROM t), + * (SELECT sum(b) FROM t) + * + * is optimized from: + * + * == Optimized Logical Plan == + * Project [scalar-subquery#242 [] AS scalarsubquery()#253, + * scalar-subquery#243 [] AS scalarsubquery()#254L] + * : :- Aggregate [avg(a#244) AS avg(a)#247] + * : : +- Project [a#244] + * : : +- Relation default.t[a#244,b#245] parquet + * : +- Aggregate [sum(a#251) AS sum(a)#250L] + * : +- Project [a#251] + * : +- Relation default.t[a#251,b#252] parquet + * +- OneRowRelation + * + * to: + * + * == Optimized Logical Plan == + * Project [scalar-subquery#242 [].avg(a) AS scalarsubquery()#253, + * scalar-subquery#243 [].sum(a) AS scalarsubquery()#254L] + * : :- Project [named_struct(avg(a), avg(a)#247, sum(a), sum(a)#250L) AS mergedValue#260] + * : : +- Aggregate [avg(a#244) AS avg(a)#247, sum(a#244) AS sum(a)#250L] + * : : +- Project [a#244] + * : : +- Relation default.t[a#244,b#245] parquet + * : +- Project [named_struct(avg(a), avg(a)#247, sum(a), sum(a)#250L) AS mergedValue#260] + * : +- Aggregate [avg(a#244) AS avg(a)#247, sum(a#244) AS sum(a)#250L] + * : +- Project [a#244] + * : +- Relation default.t[a#244,b#245] parquet + * +- OneRowRelation + * + * == Physical Plan == + * *(1) Project [Subquery scalar-subquery#242, [id=#125].avg(a) AS scalarsubquery()#253, + * ReusedSubquery + * Subquery scalar-subquery#242, [id=#125].sum(a) AS scalarsubquery()#254L] + * : :- Subquery scalar-subquery#242, [id=#125] + * : : +- *(2) Project [named_struct(avg(a), avg(a)#247, sum(a), sum(a)#250L) AS mergedValue#260] + * : : +- *(2) HashAggregate(keys=[], functions=[avg(a#244), sum(a#244)], + * output=[avg(a)#247, sum(a)#250L]) + * : : +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#120] + * : : +- *(1) HashAggregate(keys=[], functions=[partial_avg(a#244), partial_sum(a#244)], + * output=[sum#262, count#263L, sum#264L]) + * : : +- *(1) ColumnarToRow + * : : +- FileScan parquet default.t[a#244] ... + * : +- ReusedSubquery Subquery scalar-subquery#242, [id=#125] + * +- *(1) Scan OneRowRelation[] + */ +object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper { + def apply(plan: LogicalPlan): LogicalPlan = { + plan match { + // Subquery reuse needs to be enabled for this optimization. + case _ if !conf.getConf(SQLConf.SUBQUERY_REUSE_ENABLED) => plan + + // This rule does a whole plan traversal, no need to run on subqueries. + case _: Subquery => plan + + // Plans with CTEs are not supported for now. + case _: WithCTE => plan + + case _ => extractCommonScalarSubqueries(plan) + } + } + + /** + * An item in the cache of merged scalar subqueries. + * + * @param attributes Attributes that form the struct scalar return value of a merged subquery. + * @param plan The plan of a merged scalar subquery. + * @param merged A flag to identify if this item is the result of merging subqueries. + * Please note that `attributes.size == 1` doesn't always mean that the plan is not + * merged as there can be subqueries that are different ([[checkIdenticalPlans]] is + * false) due to an extra [[Project]] node in one of them. In that case + * `attributes.size` remains 1 after merging, but the merged flag becomes true. + */ + case class Header(attributes: Seq[Attribute], plan: LogicalPlan, merged: Boolean) + + private def extractCommonScalarSubqueries(plan: LogicalPlan) = { + val cache = ArrayBuffer.empty[Header] + val planWithReferences = insertReferences(plan, cache) + cache.zipWithIndex.foreach { case (header, i) => + cache(i) = cache(i).copy(plan = + if (header.merged) { + CTERelationDef( + createProject(header.attributes, removeReferences(header.plan, cache)), + underSubquery = true) + } else { + removeReferences(header.plan, cache) + }) + } + val newPlan = removeReferences(planWithReferences, cache) + val subqueryCTEs = cache.filter(_.merged).map(_.plan.asInstanceOf[CTERelationDef]) + if (subqueryCTEs.nonEmpty) { + WithCTE(newPlan, subqueryCTEs.toSeq) + } else { + newPlan + } + } + + // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan. + private def insertReferences(plan: LogicalPlan, cache: ArrayBuffer[Header]): LogicalPlan = { + plan.transformUpWithSubqueries { + case n => n.transformExpressionsUpWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) { + case s: ScalarSubquery if !s.isCorrelated && s.deterministic => + val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache) + ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId) + } + } + } + + // Caching returns the index of the subquery in the cache and the index of scalar member in the + // "Header". + private def cacheSubquery(plan: LogicalPlan, cache: ArrayBuffer[Header]): (Int, Int) = { + val output = plan.output.head + cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) => + checkIdenticalPlans(plan, header.plan).map { outputMap => + val mappedOutput = mapAttributes(output, outputMap) + val headerIndex = header.attributes.indexWhere(_.exprId == mappedOutput.exprId) + subqueryIndex -> headerIndex + }.orElse(tryMergePlans(plan, header.plan).map { + case (mergedPlan, outputMap) => + val mappedOutput = mapAttributes(output, outputMap) + var headerIndex = header.attributes.indexWhere(_.exprId == mappedOutput.exprId) + val newHeaderAttributes = if (headerIndex == -1) { + headerIndex = header.attributes.size + header.attributes :+ mappedOutput + } else { + header.attributes + } + cache(subqueryIndex) = Header(newHeaderAttributes, mergedPlan, true) + subqueryIndex -> headerIndex + }) + }).getOrElse { + cache += Header(Seq(output), plan, false) + cache.length - 1 -> 0 + } + } + + // If 2 plans are identical return the attribute mapping from the new to the cached version. + private def checkIdenticalPlans( + newPlan: LogicalPlan, + cachedPlan: LogicalPlan): Option[AttributeMap[Attribute]] = { + if (newPlan.canonicalized == cachedPlan.canonicalized) { + Some(AttributeMap(newPlan.output.zip(cachedPlan.output))) + } else { + None + } + } + + // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged + // plan with the attribute mapping from the new to the merged version. + // Please note that merging arbitrary plans can be complicated, the current version supports only + // some of the most important nodes. + private def tryMergePlans( + newPlan: LogicalPlan, + cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = { + checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse( + (newPlan, cachedPlan) match { + case (np: Project, cp: Project) => + tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) => + val (mergedProjectList, newOutputMap) = + mergeNamedExpressions(np.projectList, outputMap, cp.projectList) + val mergedPlan = Project(mergedProjectList, mergedChild) + mergedPlan -> newOutputMap + } + case (np, cp: Project) => + tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) => + val (mergedProjectList, newOutputMap) = + mergeNamedExpressions(np.output, outputMap, cp.projectList) + val mergedPlan = Project(mergedProjectList, mergedChild) + mergedPlan -> newOutputMap + } + case (np: Project, cp) => + tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) => + val (mergedProjectList, newOutputMap) = + mergeNamedExpressions(np.projectList, outputMap, cp.output) + val mergedPlan = Project(mergedProjectList, mergedChild) + mergedPlan -> newOutputMap + } + case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) => + tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) => + val mappedNewGroupingExpression = + np.groupingExpressions.map(mapAttributes(_, outputMap)) + // Order of grouping expression does matter as merging different grouping orders can + // introduce "extra" shuffles/sorts that might not present in all of the original + // subqueries. + if (mappedNewGroupingExpression.map(_.canonicalized) == + cp.groupingExpressions.map(_.canonicalized)) { + val (mergedAggregateExpressions, newOutputMap) = + mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions) + val mergedPlan = + Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild) + Some(mergedPlan -> newOutputMap) + } else { + None + } + } + + case (np: Filter, cp: Filter) => + tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) => + val mappedNewCondition = mapAttributes(np.condition, outputMap) + // Comparing the canonicalized form is required to ignore different forms of the same + // expression. + if (mappedNewCondition.canonicalized == cp.condition.canonicalized) { + val mergedPlan = cp.withNewChildren(Seq(mergedChild)) + Some(mergedPlan -> outputMap) + } else { + None + } + } + + case (np: Join, cp: Join) if np.joinType == cp.joinType && np.hint == cp.hint => + tryMergePlans(np.left, cp.left).flatMap { case (mergedLeft, leftOutputMap) => + tryMergePlans(np.right, cp.right).flatMap { case (mergedRight, rightOutputMap) => + val outputMap = leftOutputMap ++ rightOutputMap + val mappedNewCondition = np.condition.map(mapAttributes(_, outputMap)) + // Comparing the canonicalized form is required to ignore different forms of the same + // expression and `AttributeReference.quailifier`s in `cp.condition`. + if (mappedNewCondition.map(_.canonicalized) == cp.condition.map(_.canonicalized)) { + val mergedPlan = cp.withNewChildren(Seq(mergedLeft, mergedRight)) + Some(mergedPlan -> outputMap) + } else { + None + } + } + } + + // Otherwise merging is not possible. + case _ => None + }) + } + + private def createProject(attributes: Seq[Attribute], plan: LogicalPlan): Project = { + Project( + Seq(Alias( + CreateNamedStruct(attributes.flatMap(a => Seq(Literal(a.name), a))), + "mergedValue")()), + plan) + } + + private def mapAttributes[T <: Expression](expr: T, outputMap: AttributeMap[Attribute]) = { + expr.transform { + case a: Attribute => outputMap.getOrElse(a, a) + }.asInstanceOf[T] + } + + // Applies `outputMap` attribute mapping on attributes of `newExpressions` and merges them into + // `cachedExpressions`. Returns the merged expressions and the attribute mapping from the new to + // the merged version that can be propagated up during merging nodes. + private def mergeNamedExpressions( + newExpressions: Seq[NamedExpression], + outputMap: AttributeMap[Attribute], + cachedExpressions: Seq[NamedExpression]) = { + val mergedExpressions = ArrayBuffer[NamedExpression](cachedExpressions: _*) + val newOutputMap = AttributeMap(newExpressions.map { ne => + val mapped = mapAttributes(ne, outputMap) + val withoutAlias = mapped match { + case Alias(child, _) => child + case e => e + } + ne.toAttribute -> mergedExpressions.find { + case Alias(child, _) => child semanticEquals withoutAlias + case e => e semanticEquals withoutAlias + }.getOrElse { + mergedExpressions += mapped + mapped + }.toAttribute + }) + (mergedExpressions.toSeq, newOutputMap) + } + + // Only allow aggregates of the same implementation because merging different implementations + // could cause performance regression. + private def supportedAggregateMerge(newPlan: Aggregate, cachedPlan: Aggregate) = { + val newPlanAggregateExpressions = newPlan.aggregateExpressions.flatMap(_.collect { + case a: AggregateExpression => a + }) + val cachedPlanAggregateExpressions = cachedPlan.aggregateExpressions.flatMap(_.collect { + case a: AggregateExpression => a + }) + val newPlanSupportsHashAggregate = Aggregate.supportsHashAggregate( + newPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)) + val cachedPlanSupportsHashAggregate = Aggregate.supportsHashAggregate( + cachedPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)) + newPlanSupportsHashAggregate && cachedPlanSupportsHashAggregate || + newPlanSupportsHashAggregate == cachedPlanSupportsHashAggregate && { + val newPlanSupportsObjectHashAggregate = + Aggregate.supportsObjectHashAggregate(newPlanAggregateExpressions) + val cachedPlanSupportsObjectHashAggregate = + Aggregate.supportsObjectHashAggregate(cachedPlanAggregateExpressions) + newPlanSupportsObjectHashAggregate && cachedPlanSupportsObjectHashAggregate || + newPlanSupportsObjectHashAggregate == cachedPlanSupportsObjectHashAggregate + } + } + + // Second traversal replaces `ScalarSubqueryReference`s to either + // `GetStructField(ScalarSubquery(CTERelationRef to the merged plan)` if the plan is merged from + // multiple subqueries or `ScalarSubquery(original plan)` if it isn't. + private def removeReferences( + plan: LogicalPlan, + cache: ArrayBuffer[Header]) = { + plan.transformUpWithSubqueries { + case n => + n.transformExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY_REFERENCE)) { + case ssr: ScalarSubqueryReference => + val header = cache(ssr.subqueryIndex) + if (header.merged) { + val subqueryCTE = header.plan.asInstanceOf[CTERelationDef] + GetStructField( + ScalarSubquery( + CTERelationRef(subqueryCTE.id, _resolved = true, subqueryCTE.output), + exprId = ssr.exprId), + ssr.headerIndex) + } else { + ScalarSubquery(header.plan, exprId = ssr.exprId) + } + } + } + } +} + +/** + * Temporal reference to a subquery. + */ +case class ScalarSubqueryReference( + subqueryIndex: Int, + headerIndex: Int, + dataType: DataType, + exprId: ExprId) extends LeafExpression with Unevaluable { + override def nullable: Boolean = true + + final override val nodePatterns: Seq[TreePattern] = Seq(SCALAR_SUBQUERY_REFERENCE) + + override def stringArgs: Iterator[Any] = Iterator(subqueryIndex, headerIndex, dataType, exprId.id) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala index ab9f20edb0bb9..2195eef2fc93b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala @@ -121,7 +121,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { private def pushdownPredicatesAndAttributes( plan: LogicalPlan, cteMap: CTEMap): LogicalPlan = plan.transformWithSubqueries { - case cteDef @ CTERelationDef(child, id, originalPlanWithPredicates) => + case cteDef @ CTERelationDef(child, id, originalPlanWithPredicates, _) => val (_, _, newPreds, newAttrSet) = cteMap(id) val originalPlan = originalPlanWithPredicates.map(_._1).getOrElse(child) val preds = originalPlanWithPredicates.map(_._2).getOrElse(Seq.empty) @@ -169,7 +169,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { object CleanUpTempCTEInfo extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(CTE)) { - case cteDef @ CTERelationDef(_, _, Some(_)) => + case cteDef @ CTERelationDef(_, _, Some(_), _) => cteDef.copy(originalPlanWithPredicates = None) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceCTERefWithRepartition.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceCTERefWithRepartition.scala index e0d0417ce5161..0190fa2a2ab09 100644 --- 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 @@ -47,13 +47,14 @@ object ReplaceCTERefWithRepartition extends Rule[LogicalPlan] { 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) - } + val withRepartition = + if (inlined.isInstanceOf[RepartitionOperation] || cteDef.underSubquery) { + // If the CTE definition plan itself is a repartition operation or if it hosts a merged + // scalar subquery, 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) 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 07066a4bb328b..419e28c800791 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 @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.analysis.{AnsiTypeCoercion, MultiInstanceRe import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.catalog.CatalogTable.VIEW_STORING_ANALYZED_PLAN import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, TypedImperativeAggregate} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning, SinglePartition} import org.apache.spark.sql.catalyst.trees.TreeNodeTag @@ -663,11 +663,14 @@ case class UnresolvedWith( * 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. + * @param underSubquery If true, it means we don't need to add a shuffle for this CTE relation as + * subquery reuse will be applied to reuse CTE relation output. */ case class CTERelationDef( child: LogicalPlan, id: Long = CTERelationDef.newId, - originalPlanWithPredicates: Option[(LogicalPlan, Seq[Expression])] = None) extends UnaryNode { + originalPlanWithPredicates: Option[(LogicalPlan, Seq[Expression])] = None, + underSubquery: Boolean = false) extends UnaryNode { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) @@ -678,17 +681,19 @@ case class CTERelationDef( } object CTERelationDef { - private val curId = new java.util.concurrent.atomic.AtomicLong() + private[sql] val curId = new java.util.concurrent.atomic.AtomicLong() def newId: Long = curId.getAndIncrement() } /** * Represents the relation of a CTE reference. - * @param cteId The ID of the corresponding CTE definition. - * @param _resolved Whether this reference is resolved. - * @param output The output attributes of this CTE reference, which can be different from - * the output of its corresponding CTE definition after attribute de-duplication. - * @param statsOpt The optional statistics inferred from the corresponding CTE definition. + * @param cteId The ID of the corresponding CTE definition. + * @param _resolved Whether this reference is resolved. + * @param output The output attributes of this CTE reference, which can be different + * from the output of its corresponding CTE definition after attribute + * de-duplication. + * @param statsOpt The optional statistics inferred from the corresponding CTE + * definition. */ case class CTERelationRef( cteId: Long, @@ -1014,6 +1019,24 @@ case class Aggregate( } } +object Aggregate { + def isAggregateBufferMutable(schema: StructType): Boolean = { + schema.forall(f => UnsafeRow.isMutable(f.dataType)) + } + + def supportsHashAggregate(aggregateBufferAttributes: Seq[Attribute]): Boolean = { + val aggregationBufferSchema = StructType.fromAttributes(aggregateBufferAttributes) + isAggregateBufferMutable(aggregationBufferSchema) + } + + def supportsObjectHashAggregate(aggregateExpressions: Seq[AggregateExpression]): Boolean = { + aggregateExpressions.map(_.aggregateFunction).exists { + case _: TypedImperativeAggregate[_] => true + case _ => false + } + } +} + case class Window( windowExpressions: Seq[NamedExpression], partitionSpec: Seq[Expression], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 3cf45d5f79f00..93273b5a2c7a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -77,6 +77,7 @@ object TreePattern extends Enumeration { val REGEXP_REPLACE: Value = Value val RUNTIME_REPLACEABLE: Value = Value val SCALAR_SUBQUERY: Value = Value + val SCALAR_SUBQUERY_REFERENCE: Value = Value val SCALA_UDF: Value = Value val SORT: Value = Value val SUBQUERY_ALIAS: Value = Value diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala new file mode 100644 index 0000000000000..8af0e02855b12 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala @@ -0,0 +1,578 @@ +/* + * 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 org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateNamedStruct, GetStructField, Literal, ScalarSubquery} +import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, CollectSet} +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + +class MergeScalarSubqueriesSuite extends PlanTest { + + override def beforeEach(): Unit = { + CTERelationDef.curId.set(0) + } + + private object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("MergeScalarSubqueries", Once, MergeScalarSubqueries) :: Nil + } + + val testRelation = LocalRelation('a.int, 'b.int, 'c.string) + + private def definitionNode(plan: LogicalPlan, cteIndex: Int) = { + CTERelationDef(plan, cteIndex, underSubquery = true) + } + + private def extractorExpression(cteIndex: Int, output: Seq[Attribute], fieldIndex: Int) = { + GetStructField(ScalarSubquery(CTERelationRef(cteIndex, _resolved = true, output)), fieldIndex) + .as("scalarsubquery()") + } + + test("Merging subqueries with projects") { + val subquery1 = ScalarSubquery(testRelation.select(('a + 1).as("a_plus1"))) + val subquery2 = ScalarSubquery(testRelation.select(('a + 2).as("a_plus2"))) + val subquery3 = ScalarSubquery(testRelation.select('b)) + val subquery4 = ScalarSubquery(testRelation.select(('a + 1).as("a_plus1_2"))) + val subquery5 = ScalarSubquery(testRelation.select(('a + 2).as("a_plus2_2"))) + val subquery6 = ScalarSubquery(testRelation.select('b.as("b_2"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2, + subquery3, + subquery4, + subquery5, + subquery6) + + val mergedSubquery = testRelation + .select( + ('a + 1).as("a_plus1"), + ('a + 2).as("a_plus2"), + 'b) + .select( + CreateNamedStruct(Seq( + Literal("a_plus1"), 'a_plus1, + Literal("a_plus2"), 'a_plus2, + Literal("b"), 'b + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1), + extractorExpression(0, analyzedMergedSubquery.output, 2), + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1), + extractorExpression(0, analyzedMergedSubquery.output, 2)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Merging subqueries with aggregates") { + val subquery1 = ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a"))) + val subquery3 = ScalarSubquery(testRelation.groupBy('b)('b)) + val subquery4 = ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a_2"))) + val subquery5 = ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a_2"))) + val subquery6 = ScalarSubquery(testRelation.groupBy('b)('b.as("b_2"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2, + subquery3, + subquery4, + subquery5, + subquery6) + + val mergedSubquery = testRelation + .groupBy('b)( + max('a).as("max_a"), + sum('a).as("sum_a"), + 'b) + .select(CreateNamedStruct(Seq( + Literal("max_a"), 'max_a, + Literal("sum_a"), 'sum_a, + Literal("b"), 'b + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1), + extractorExpression(0, analyzedMergedSubquery.output, 2), + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1), + extractorExpression(0, analyzedMergedSubquery.output, 2)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Merging subqueries with aggregates with complex grouping expressions") { + val subquery1 = ScalarSubquery(testRelation.groupBy('b > 1 && 'a === 2)(max('a).as("max_a"))) + val subquery2 = ScalarSubquery( + testRelation + .select('a, 'b.as("b_2")) + .groupBy(Literal(2) === 'a && Literal(1) < 'b_2)(sum('a).as("sum_a"))) + + val originalQuery = testRelation + .select( + subquery1, + subquery2) + + val mergedSubquery = testRelation + .select('a, 'b, 'c) + .groupBy('b > 1 && 'a === 2)( + max('a).as("max_a"), + sum('a).as("sum_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), 'max_a, + Literal("sum_a"), 'sum_a + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Merging subqueries with aggregates with multiple grouping expressions") { + // supports HashAggregate + val subquery1 = ScalarSubquery(testRelation.groupBy('b, 'c)(max('a).as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.groupBy('b, 'c)(min('a).as("min_a"))) + + val originalQuery = testRelation + .select( + subquery1, + subquery2) + + val hashAggregates = testRelation + .groupBy('b, 'c)( + max('a).as("max_a"), + min('a).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), 'max_a, + Literal("min_a"), 'min_a + )).as("mergedValue")) + val analyzedHashAggregates = hashAggregates.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedHashAggregates.output, 0), + extractorExpression(0, analyzedHashAggregates.output, 1)), + Seq(definitionNode(analyzedHashAggregates, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Merging subqueries with filters") { + val subquery1 = ScalarSubquery(testRelation.where('a > 1).select('a)) + // Despite having an extra Project node, `subquery2` is mergeable with `subquery1` + val subquery2 = ScalarSubquery(testRelation.where('a > 1).select('b.as("b_1")).select('b_1)) + // Despite lacking a Project node, `subquery3` is mergeable with the result of merging + // `subquery1` and `subquery2` + val subquery3 = ScalarSubquery(testRelation.select('a.as("a_2")).where('a_2 > 1).select('a_2)) + val subquery4 = ScalarSubquery( + testRelation.select('a.as("a_2"), 'b).where('a_2 > 1).select('b.as("b_2"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2, + subquery3, + subquery4) + + val mergedSubquery = testRelation + .select('a, 'b, 'c) + .where('a > 1) + .select('a, 'b, 'c) + .select('a, 'b) + .select(CreateNamedStruct(Seq( + Literal("a"), 'a, + Literal("b"), 'b + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1), + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Merging subqueries with complex filter conditions") { + val subquery1 = ScalarSubquery(testRelation.where('a > 1 && 'b === 2).select('a)) + val subquery2 = ScalarSubquery( + testRelation + .select('a.as("a_2"), 'b) + .where(Literal(2) === 'b && Literal(1) < 'a_2) + .select('b.as("b_2"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2) + + val mergedSubquery = testRelation + .select('a, 'b, 'c) + .where('a > 1 && 'b === 2) + .select('a, 'b.as("b_2")) + .select(CreateNamedStruct(Seq( + Literal("a"), 'a, + Literal("b_2"), 'b_2 + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Do not merge subqueries with different filter conditions") { + val subquery1 = ScalarSubquery(testRelation.where('a > 1).select('a)) + val subquery2 = ScalarSubquery(testRelation.where('a < 1).select('a)) + + val originalQuery = testRelation + .select( + subquery1, + subquery2) + + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } + + test("Merging subqueries with aggregate filters") { + val subquery1 = ScalarSubquery( + testRelation.having('b)(max('a).as("max_a"))(max('a) > 1)) + val subquery2 = ScalarSubquery( + testRelation.having('b)(sum('a).as("sum_a"))(max('a) > 1)) + val originalQuery = testRelation.select( + subquery1, + subquery2) + + val mergedSubquery = testRelation + .having('b)( + max('a).as("max_a"), + sum('a).as("sum_a"))('max_a > 1) + .select( + 'max_a, + 'sum_a) + .select(CreateNamedStruct(Seq( + Literal("max_a"), 'max_a, + Literal("sum_a"), 'sum_a + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Merging subqueries with joins") { + val subquery1 = ScalarSubquery(testRelation.as("t1") + .join( + testRelation.as("t2"), + Inner, + Some($"t1.b" === $"t2.b")) + .select($"t1.a").analyze) + val subquery2 = ScalarSubquery(testRelation.as("t1") + .select('a.as("a_1"), 'b.as("b_1"), 'c.as("c_1")) + .join( + testRelation.as("t2").select('a.as("a_2"), 'b.as("b_2"), 'c.as("c_2")), + Inner, + Some('b_1 === 'b_2)) + .select('c_2).analyze) + val originalQuery = testRelation.select( + subquery1, + subquery2) + + val mergedSubquery = testRelation.as("t1") + .select('a, 'b, 'c) + .join( + testRelation.as("t2").select('a, 'b, 'c), + Inner, + Some($"t1.b" === $"t2.b")) + .select($"t1.a", $"t2.c") + .select(CreateNamedStruct(Seq( + Literal("a"), 'a, + Literal("c"), 'c + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Merge subqueries with complex join conditions") { + val subquery1 = ScalarSubquery(testRelation.as("t1") + .join( + testRelation.as("t2"), + Inner, + Some($"t1.b" < $"t2.b" && $"t1.a" === $"t2.c")) + .select($"t1.a").analyze) + val subquery2 = ScalarSubquery(testRelation.as("t1") + .select('a.as("a_1"), 'b.as("b_1"), 'c.as("c_1")) + .join( + testRelation.as("t2").select('a.as("a_2"), 'b.as("b_2"), 'c.as("c_2")), + Inner, + Some('c_2 === 'a_1 && 'b_1 < 'b_2)) + .select('c_2).analyze) + val originalQuery = testRelation.select( + subquery1, + subquery2) + + val mergedSubquery = testRelation.as("t1") + .select('a, 'b, 'c) + .join( + testRelation.as("t2").select('a, 'b, 'c), + Inner, + Some($"t1.b" < $"t2.b" && $"t1.a" === $"t2.c")) + .select($"t1.a", $"t2.c") + .select(CreateNamedStruct(Seq( + Literal("a"), 'a, + Literal("c"), 'c + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Do not merge subqueries with different join types") { + val subquery1 = ScalarSubquery(testRelation.as("t1") + .join( + testRelation.as("t2"), + Inner, + Some($"t1.b" === $"t2.b")) + .select($"t1.a")) + val subquery2 = ScalarSubquery(testRelation.as("t1") + .join( + testRelation.as("t2"), + LeftOuter, + Some($"t1.b" === $"t2.b")) + .select($"t1.a")) + val originalQuery = testRelation.select( + subquery1, + subquery2) + + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } + + test("Do not merge subqueries with different join conditions") { + val subquery1 = ScalarSubquery(testRelation.as("t1") + .join( + testRelation.as("t2"), + Inner, + Some($"t1.b" < $"t2.b")) + .select($"t1.a")) + val subquery2 = ScalarSubquery(testRelation.as("t1") + .join( + testRelation.as("t2"), + Inner, + Some($"t1.b" > $"t2.b")) + .select($"t1.a")) + val originalQuery = testRelation.select( + subquery1, + subquery2) + + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } + + test("Do not merge subqueries with nondeterministic elements") { + val subquery1 = ScalarSubquery(testRelation.select(('a + rand(0)).as("rand_a"))) + val subquery2 = ScalarSubquery(testRelation.select(('b + rand(0)).as("rand_b"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2) + + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + + val subquery3 = ScalarSubquery(testRelation.where('a > rand(0)).select('a)) + val subquery4 = ScalarSubquery(testRelation.where('a > rand(0)).select('b)) + val originalQuery2 = testRelation + .select( + subquery3, + subquery4) + + comparePlans(Optimize.execute(originalQuery2.analyze), originalQuery2.analyze) + + val subquery5 = ScalarSubquery(testRelation.groupBy()((max('a) + rand(0)).as("max_a"))) + val subquery6 = ScalarSubquery(testRelation.groupBy()((max('b) + rand(0)).as("max_b"))) + val originalQuery3 = testRelation + .select( + subquery5, + subquery6) + + comparePlans(Optimize.execute(originalQuery3.analyze), originalQuery3.analyze) + } + + test("Do not merge different aggregate implementations") { + // supports HashAggregate + val subquery1 = ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.groupBy('b)(min('a).as("min_a"))) + + // supports ObjectHashAggregate + val subquery3 = ScalarSubquery(testRelation + .groupBy('b)(CollectList('a).toAggregateExpression(isDistinct = false).as("collectlist_a"))) + val subquery4 = ScalarSubquery(testRelation + .groupBy('b)(CollectSet('a).toAggregateExpression(isDistinct = false).as("collectset_a"))) + + // supports SortAggregate + val subquery5 = ScalarSubquery(testRelation.groupBy('b)(max('c).as("max_c"))) + val subquery6 = ScalarSubquery(testRelation.groupBy('b)(min('c).as("min_c"))) + + val originalQuery = testRelation + .select( + subquery1, + subquery2, + subquery3, + subquery4, + subquery5, + subquery6) + + val hashAggregates = testRelation + .groupBy('b)( + max('a).as("max_a"), + min('a).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), 'max_a, + Literal("min_a"), 'min_a + )).as("mergedValue")) + val analyzedHashAggregates = hashAggregates.analyze + val objectHashAggregates = testRelation + .groupBy('b)( + CollectList('a).toAggregateExpression(isDistinct = false).as("collectlist_a"), + CollectSet('a).toAggregateExpression(isDistinct = false).as("collectset_a")) + .select(CreateNamedStruct(Seq( + Literal("collectlist_a"), 'collectlist_a, + Literal("collectset_a"), 'collectset_a + )).as("mergedValue")) + val analyzedObjectHashAggregates = objectHashAggregates.analyze + val sortAggregates = testRelation + .groupBy('b)( + max('c).as("max_c"), + min('c).as("min_c")) + .select(CreateNamedStruct(Seq( + Literal("max_c"), 'max_c, + Literal("min_c"), 'min_c + )).as("mergedValue")) + val analyzedSortAggregates = sortAggregates.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedHashAggregates.output, 0), + extractorExpression(0, analyzedHashAggregates.output, 1), + extractorExpression(1, analyzedObjectHashAggregates.output, 0), + extractorExpression(1, analyzedObjectHashAggregates.output, 1), + extractorExpression(2, analyzedSortAggregates.output, 0), + extractorExpression(2, analyzedSortAggregates.output, 1)), + Seq( + definitionNode(analyzedHashAggregates, 0), + definitionNode(analyzedObjectHashAggregates, 1), + definitionNode(analyzedSortAggregates, 2))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Do not merge subqueries with different aggregate grouping orders") { + // supports HashAggregate + val subquery1 = ScalarSubquery(testRelation.groupBy('b, 'c)(max('a).as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.groupBy('c, 'b)(min('a).as("min_a"))) + + val originalQuery = testRelation + .select( + subquery1, + subquery2) + + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } + + test("Merging subqueries from different places") { + val subquery1 = ScalarSubquery(testRelation.select(('a + 1).as("a_plus1"))) + val subquery2 = ScalarSubquery(testRelation.select(('a + 2).as("a_plus2"))) + val subquery3 = ScalarSubquery(testRelation.select('b)) + val subquery4 = ScalarSubquery(testRelation.select(('a + 1).as("a_plus1_2"))) + val subquery5 = ScalarSubquery(testRelation.select(('a + 2).as("a_plus2_2"))) + val subquery6 = ScalarSubquery(testRelation.select('b.as("b_2"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2, + subquery3) + .where( + subquery4 + + subquery5 + + subquery6 === 0) + + val mergedSubquery = testRelation + .select( + ('a + 1).as("a_plus1"), + ('a + 2).as("a_plus2"), + 'b) + .select( + CreateNamedStruct(Seq( + Literal("a_plus1"), 'a_plus1, + Literal("a_plus2"), 'a_plus2, + Literal("b"), 'b + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1), + extractorExpression(0, analyzedMergedSubquery.output, 2)) + .where( + extractorExpression(0, analyzedMergedSubquery.output, 0) + + extractorExpression(0, analyzedMergedSubquery.output, 1) + + extractorExpression(0, analyzedMergedSubquery.output, 2) === 0), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java index 31e10af38a42b..8587d9290078b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; -import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.catalyst.plans.logical.Aggregate$; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.KVIterator; import org.apache.spark.unsafe.Platform; @@ -68,12 +68,7 @@ public final class UnsafeFixedWidthAggregationMap { * schema, false otherwise. */ public static boolean supportsAggregationBufferSchema(StructType schema) { - for (StructField field: schema.fields()) { - if (!UnsafeRow.isMutable(field.dataType())) { - return false; - } - } - return true; + return Aggregate$.MODULE$.isAggregateBufferMutable(schema); } /** 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 d9457a20d91c9..84e5975189b8f 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 @@ -54,6 +54,8 @@ class SparkOptimizer( Batch("InjectRuntimeFilter", FixedPoint(1), InjectRuntimeFilter, RewritePredicateSubquery) :+ + Batch("MergeScalarSubqueries", Once, + MergeScalarSubqueries) :+ Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala index 26161acae30b2..3e5846bcdfd77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.aggregate import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.plans.logical.Aggregate import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf @@ -73,7 +74,7 @@ object AggUtils { initialInputBufferOffset: Int = 0, resultExpressions: Seq[NamedExpression] = Nil, child: SparkPlan): SparkPlan = { - val useHash = HashAggregateExec.supportsAggregate( + val useHash = Aggregate.supportsHashAggregate( aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)) val forceSortAggregate = forceApplySortAggregate(child.conf) @@ -90,7 +91,7 @@ object AggUtils { child = child) } else { val objectHashEnabled = child.conf.useObjectHashAggregation - val useObjectHash = ObjectHashAggregateExec.supportsAggregate(aggregateExpressions) + val useObjectHash = Aggregate.supportsObjectHashAggregate(aggregateExpressions) if (objectHashEnabled && useObjectHash && !forceSortAggregate) { ObjectHashAggregateExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 8be3a018cee58..6c83ba5546d2a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.plans.logical.Aggregate import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_MILLIS import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution._ @@ -55,7 +56,7 @@ case class HashAggregateExec( child: SparkPlan) extends AggregateCodegenSupport { - require(HashAggregateExec.supportsAggregate(aggregateBufferAttributes)) + require(Aggregate.supportsHashAggregate(aggregateBufferAttributes)) override lazy val allAttributes: AttributeSeq = child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++ @@ -885,10 +886,3 @@ case class HashAggregateExec( override protected def withNewChildInternal(newChild: SparkPlan): HashAggregateExec = copy(child = newChild) } - -object HashAggregateExec { - def supportsAggregate(aggregateBufferAttributes: Seq[Attribute]): Boolean = { - val aggregationBufferSchema = StructType.fromAttributes(aggregateBufferAttributes) - UnsafeFixedWidthAggregationMap.supportsAggregationBufferSchema(aggregationBufferSchema) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index 9da0ca93c1819..e6530e94701f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -143,12 +143,3 @@ case class ObjectHashAggregateExec( override protected def withNewChildInternal(newChild: SparkPlan): ObjectHashAggregateExec = copy(child = newChild) } - -object ObjectHashAggregateExec { - def supportsAggregate(aggregateExpressions: Seq[AggregateExpression]): Boolean = { - aggregateExpressions.map(_.aggregateFunction).exists { - case _: TypedImperativeAggregate[_] => true - case _ => false - } - } -} diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt index 8736c9861a5ce..8bf63794f25e4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt @@ -20,699 +20,284 @@ Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) (4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3] > 62316685) THEN Subquery scalar-subquery#4, [id=#5] ELSE Subquery scalar-subquery#6, [id=#7] END AS bucket1#8, CASE WHEN (Subquery scalar-subquery#9, [id=#10] > 19045798) THEN Subquery scalar-subquery#11, [id=#12] ELSE Subquery scalar-subquery#13, [id=#14] END AS bucket2#15, CASE WHEN (Subquery scalar-subquery#16, [id=#17] > 365541424) THEN Subquery scalar-subquery#18, [id=#19] ELSE Subquery scalar-subquery#20, [id=#21] END AS bucket3#22, CASE WHEN (Subquery scalar-subquery#23, [id=#24] > 216357808) THEN Subquery scalar-subquery#25, [id=#26] ELSE Subquery scalar-subquery#27, [id=#28] END AS bucket4#29, CASE WHEN (Subquery scalar-subquery#30, [id=#31] > 184483884) THEN Subquery scalar-subquery#32, [id=#33] ELSE Subquery scalar-subquery#34, [id=#35] END AS bucket5#36] +Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* HashAggregate (11) -+- Exchange (10) - +- * HashAggregate (9) - +- * Project (8) - +- * Filter (7) - +- * ColumnarToRow (6) - +- Scan parquet default.store_sales (5) +* Project (12) ++- * HashAggregate (11) + +- Exchange (10) + +- * HashAggregate (9) + +- * Project (8) + +- * Filter (7) + +- * ColumnarToRow (6) + +- Scan parquet default.store_sales (5) (5) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct +ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] (7) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) (8) Project [codegen id : 1] -Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_ext_discount_amt#18, ss_net_paid#19] +Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] (9) HashAggregate [codegen id : 1] -Input: [] +Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#39] -Results [1]: [count#40] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] +Aggregate Attributes [5]: [count#21, sum#22, count#23, sum#24, count#25] +Results [5]: [count#26, sum#27, count#28, sum#29, count#30] (10) Exchange -Input [1]: [count#40] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#41] +Input [5]: [count#26, sum#27, count#28, sum#29, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (11) HashAggregate [codegen id : 2] -Input [1]: [count#40] +Input [5]: [count#26, sum#27, count#28, sum#29, count#30] Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#42] -Results [1]: [count(1)#42 AS count(1)#43] - -Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#4, [id=#5] -* HashAggregate (18) -+- Exchange (17) - +- * HashAggregate (16) - +- * Project (15) - +- * Filter (14) - +- * ColumnarToRow (13) - +- Scan parquet default.store_sales (12) - - -(12) Scan parquet default.store_sales -Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] - -(14) Filter [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) - -(15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#45] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] - -(16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#45] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [2]: [sum#47, count#48] -Results [2]: [sum#49, count#50] - -(17) Exchange -Input [2]: [sum#49, count#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] - -(18) HashAggregate [codegen id : 2] -Input [2]: [sum#49, count#50] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] - -Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] -* HashAggregate (25) -+- Exchange (24) - +- * HashAggregate (23) - +- * Project (22) - +- * Filter (21) - +- * ColumnarToRow (20) - +- Scan parquet default.store_sales (19) - - -(19) Scan parquet default.store_sales -Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] - -(21) Filter [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] -Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) - -(22) Project [codegen id : 1] -Output [1]: [ss_net_paid#55] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] - -(23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#55] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [2]: [sum#57, count#58] -Results [2]: [sum#59, count#60] - -(24) Exchange -Input [2]: [sum#59, count#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] - -(25) HashAggregate [codegen id : 2] -Input [2]: [sum#59, count#60] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* HashAggregate (32) -+- Exchange (31) - +- * HashAggregate (30) - +- * Project (29) - +- * Filter (28) - +- * ColumnarToRow (27) - +- Scan parquet default.store_sales (26) - - -(26) Scan parquet default.store_sales -Output [2]: [ss_quantity#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] +Aggregate Attributes [3]: [count(1)#32, avg(UnscaledValue(ss_ext_discount_amt#18))#33, avg(UnscaledValue(ss_net_paid#19))#34] +Results [3]: [count(1)#32 AS count(1)#35, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#33 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#36, cast((avg(UnscaledValue(ss_net_paid#19))#34 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#37] -(27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +(12) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#35, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#36, avg(ss_net_paid), avg(ss_net_paid)#37) AS mergedValue#38] +Input [3]: [count(1)#35, avg(ss_ext_discount_amt)#36, avg(ss_net_paid)#37] -(28) Filter [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] -Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) +Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(29) Project [codegen id : 1] -Output: [] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(30) HashAggregate [codegen id : 1] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#66] -Results [1]: [count#67] +Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] +* Project (20) ++- * HashAggregate (19) + +- Exchange (18) + +- * HashAggregate (17) + +- * Project (16) + +- * Filter (15) + +- * ColumnarToRow (14) + +- Scan parquet default.store_sales (13) -(31) Exchange -Input [1]: [count#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] -(32) HashAggregate [codegen id : 2] -Input [1]: [count#67] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#69] -Results [1]: [count(1)#69 AS count(1)#70] - -Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* HashAggregate (39) -+- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * Filter (35) - +- * ColumnarToRow (34) - +- Scan parquet default.store_sales (33) - - -(33) Scan parquet default.store_sales -Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +(13) Scan parquet default.store_sales +Output [4]: [ss_quantity#39, ss_ext_discount_amt#40, ss_net_paid#41, ss_sold_date_sk#42] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct +ReadSchema: struct -(34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +(14) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#39, ss_ext_discount_amt#40, ss_net_paid#41, ss_sold_date_sk#42] -(35) Filter [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) +(15) Filter [codegen id : 1] +Input [4]: [ss_quantity#39, ss_ext_discount_amt#40, ss_net_paid#41, ss_sold_date_sk#42] +Condition : ((isnotnull(ss_quantity#39) AND (ss_quantity#39 >= 21)) AND (ss_quantity#39 <= 40)) -(36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#72] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +(16) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#40, ss_net_paid#41] +Input [4]: [ss_quantity#39, ss_ext_discount_amt#40, ss_net_paid#41, ss_sold_date_sk#42] -(37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#72] +(17) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#40, ss_net_paid#41] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [2]: [sum#74, count#75] -Results [2]: [sum#76, count#77] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#40)), partial_avg(UnscaledValue(ss_net_paid#41))] +Aggregate Attributes [5]: [count#43, sum#44, count#45, sum#46, count#47] +Results [5]: [count#48, sum#49, count#50, sum#51, count#52] -(38) Exchange -Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +(18) Exchange +Input [5]: [count#48, sum#49, count#50, sum#51, count#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#53] -(39) HashAggregate [codegen id : 2] -Input [2]: [sum#76, count#77] +(19) HashAggregate [codegen id : 2] +Input [5]: [count#48, sum#49, count#50, sum#51, count#52] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] - -Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] -* HashAggregate (46) -+- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * Filter (42) - +- * ColumnarToRow (41) - +- Scan parquet default.store_sales (40) - - -(40) Scan parquet default.store_sales -Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#40)), avg(UnscaledValue(ss_net_paid#41))] +Aggregate Attributes [3]: [count(1)#54, avg(UnscaledValue(ss_ext_discount_amt#40))#55, avg(UnscaledValue(ss_net_paid#41))#56] +Results [3]: [count(1)#54 AS count(1)#57, cast((avg(UnscaledValue(ss_ext_discount_amt#40))#55 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#58, cast((avg(UnscaledValue(ss_net_paid#41))#56 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] -(41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +(20) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#57, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#58, avg(ss_net_paid), avg(ss_net_paid)#59) AS mergedValue#60] +Input [3]: [count(1)#57, avg(ss_ext_discount_amt)#58, avg(ss_net_paid)#59] -(42) Filter [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] -Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) +Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] -(43) Project [codegen id : 1] -Output [1]: [ss_net_paid#82] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] -(44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#82] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] +* Project (28) ++- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * Project (24) + +- * Filter (23) + +- * ColumnarToRow (22) + +- Scan parquet default.store_sales (21) -(45) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] -(46) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] -* HashAggregate (53) -+- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * Filter (49) - +- * ColumnarToRow (48) - +- Scan parquet default.store_sales (47) - - -(47) Scan parquet default.store_sales -Output [2]: [ss_quantity#91, ss_sold_date_sk#92] +(21) Scan parquet default.store_sales +Output [4]: [ss_quantity#61, ss_ext_discount_amt#62, ss_net_paid#63, ss_sold_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct +ReadSchema: struct -(48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +(22) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#61, ss_ext_discount_amt#62, ss_net_paid#63, ss_sold_date_sk#64] -(49) Filter [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] -Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) +(23) Filter [codegen id : 1] +Input [4]: [ss_quantity#61, ss_ext_discount_amt#62, ss_net_paid#63, ss_sold_date_sk#64] +Condition : ((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 41)) AND (ss_quantity#61 <= 60)) -(50) Project [codegen id : 1] -Output: [] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +(24) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#62, ss_net_paid#63] +Input [4]: [ss_quantity#61, ss_ext_discount_amt#62, ss_net_paid#63, ss_sold_date_sk#64] -(51) HashAggregate [codegen id : 1] -Input: [] +(25) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#62, ss_net_paid#63] Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#93] -Results [1]: [count#94] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#62)), partial_avg(UnscaledValue(ss_net_paid#63))] +Aggregate Attributes [5]: [count#65, sum#66, count#67, sum#68, count#69] +Results [5]: [count#70, sum#71, count#72, sum#73, count#74] -(52) Exchange -Input [1]: [count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +(26) Exchange +Input [5]: [count#70, sum#71, count#72, sum#73, count#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] -(53) HashAggregate [codegen id : 2] -Input [1]: [count#94] +(27) HashAggregate [codegen id : 2] +Input [5]: [count#70, sum#71, count#72, sum#73, count#74] Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#96] -Results [1]: [count(1)#96 AS count(1)#97] - -Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] -* HashAggregate (60) -+- Exchange (59) - +- * HashAggregate (58) - +- * Project (57) - +- * Filter (56) - +- * ColumnarToRow (55) - +- Scan parquet default.store_sales (54) - - -(54) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#62)), avg(UnscaledValue(ss_net_paid#63))] +Aggregate Attributes [3]: [count(1)#76, avg(UnscaledValue(ss_ext_discount_amt#62))#77, avg(UnscaledValue(ss_net_paid#63))#78] +Results [3]: [count(1)#76 AS count(1)#79, cast((avg(UnscaledValue(ss_ext_discount_amt#62))#77 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80, cast((avg(UnscaledValue(ss_net_paid#63))#78 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#81] -(55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +(28) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#79, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#80, avg(ss_net_paid), avg(ss_net_paid)#81) AS mergedValue#82] +Input [3]: [count(1)#79, avg(ss_ext_discount_amt)#80, avg(ss_net_paid)#81] -(56) Filter [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] -Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) +Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] -(57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#99] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] -(58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#99] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [2]: [sum#101, count#102] -Results [2]: [sum#103, count#104] +Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* Project (36) ++- * HashAggregate (35) + +- Exchange (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * ColumnarToRow (30) + +- Scan parquet default.store_sales (29) -(59) Exchange -Input [2]: [sum#103, count#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] -(60) HashAggregate [codegen id : 2] -Input [2]: [sum#103, count#104] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] - -Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * Filter (63) - +- * ColumnarToRow (62) - +- Scan parquet default.store_sales (61) - - -(61) Scan parquet default.store_sales -Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] - -(63) Filter [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] -Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) - -(64) Project [codegen id : 1] -Output [1]: [ss_net_paid#109] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] - -(65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#109] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [2]: [sum#111, count#112] -Results [2]: [sum#113, count#114] - -(66) Exchange -Input [2]: [sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] - -(67) HashAggregate [codegen id : 2] -Input [2]: [sum#113, count#114] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] -* HashAggregate (74) -+- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * Filter (70) - +- * ColumnarToRow (69) - +- Scan parquet default.store_sales (68) - - -(68) Scan parquet default.store_sales -Output [2]: [ss_quantity#118, ss_sold_date_sk#119] +(29) Scan parquet default.store_sales +Output [4]: [ss_quantity#83, ss_ext_discount_amt#84, ss_net_paid#85, ss_sold_date_sk#86] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct +ReadSchema: struct -(69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +(30) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#83, ss_ext_discount_amt#84, ss_net_paid#85, ss_sold_date_sk#86] -(70) Filter [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] -Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) +(31) Filter [codegen id : 1] +Input [4]: [ss_quantity#83, ss_ext_discount_amt#84, ss_net_paid#85, ss_sold_date_sk#86] +Condition : ((isnotnull(ss_quantity#83) AND (ss_quantity#83 >= 61)) AND (ss_quantity#83 <= 80)) -(71) Project [codegen id : 1] -Output: [] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +(32) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#84, ss_net_paid#85] +Input [4]: [ss_quantity#83, ss_ext_discount_amt#84, ss_net_paid#85, ss_sold_date_sk#86] -(72) HashAggregate [codegen id : 1] -Input: [] +(33) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#84, ss_net_paid#85] Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#120] -Results [1]: [count#121] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#84)), partial_avg(UnscaledValue(ss_net_paid#85))] +Aggregate Attributes [5]: [count#87, sum#88, count#89, sum#90, count#91] +Results [5]: [count#92, sum#93, count#94, sum#95, count#96] -(73) Exchange -Input [1]: [count#121] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] +(34) Exchange +Input [5]: [count#92, sum#93, count#94, sum#95, count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] -(74) HashAggregate [codegen id : 2] -Input [1]: [count#121] +(35) HashAggregate [codegen id : 2] +Input [5]: [count#92, sum#93, count#94, sum#95, count#96] Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#123] -Results [1]: [count(1)#123 AS count(1)#124] - -Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] -* HashAggregate (81) -+- Exchange (80) - +- * HashAggregate (79) - +- * Project (78) - +- * Filter (77) - +- * ColumnarToRow (76) - +- Scan parquet default.store_sales (75) - - -(75) Scan parquet default.store_sales -Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#84)), avg(UnscaledValue(ss_net_paid#85))] +Aggregate Attributes [3]: [count(1)#98, avg(UnscaledValue(ss_ext_discount_amt#84))#99, avg(UnscaledValue(ss_net_paid#85))#100] +Results [3]: [count(1)#98 AS count(1)#101, cast((avg(UnscaledValue(ss_ext_discount_amt#84))#99 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#102, cast((avg(UnscaledValue(ss_net_paid#85))#100 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#103] -(76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +(36) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#101, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#102, avg(ss_net_paid), avg(ss_net_paid)#103) AS mergedValue#104] +Input [3]: [count(1)#101, avg(ss_ext_discount_amt)#102, avg(ss_net_paid)#103] -(77) Filter [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] -Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) +Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -(78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#126] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -(79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#126] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [2]: [sum#128, count#129] -Results [2]: [sum#130, count#131] +Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] +* Project (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * Filter (39) + +- * ColumnarToRow (38) + +- Scan parquet default.store_sales (37) -(80) Exchange -Input [2]: [sum#130, count#131] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] -(81) HashAggregate [codegen id : 2] -Input [2]: [sum#130, count#131] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] - -Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* HashAggregate (88) -+- Exchange (87) - +- * HashAggregate (86) - +- * Project (85) - +- * Filter (84) - +- * ColumnarToRow (83) - +- Scan parquet default.store_sales (82) - - -(82) Scan parquet default.store_sales -Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] - -(84) Filter [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] -Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) - -(85) Project [codegen id : 1] -Output [1]: [ss_net_paid#136] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] - -(86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#136] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [2]: [sum#138, count#139] -Results [2]: [sum#140, count#141] - -(87) Exchange -Input [2]: [sum#140, count#141] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] - -(88) HashAggregate [codegen id : 2] -Input [2]: [sum#140, count#141] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* HashAggregate (95) -+- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * Filter (91) - +- * ColumnarToRow (90) - +- Scan parquet default.store_sales (89) - - -(89) Scan parquet default.store_sales -Output [2]: [ss_quantity#145, ss_sold_date_sk#146] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] - -(91) Filter [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] -Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) - -(92) Project [codegen id : 1] -Output: [] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] - -(93) HashAggregate [codegen id : 1] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#147] -Results [1]: [count#148] - -(94) Exchange -Input [1]: [count#148] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] - -(95) HashAggregate [codegen id : 2] -Input [1]: [count#148] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#150] -Results [1]: [count(1)#150 AS count(1)#151] - -Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] -* HashAggregate (102) -+- Exchange (101) - +- * HashAggregate (100) - +- * Project (99) - +- * Filter (98) - +- * ColumnarToRow (97) - +- Scan parquet default.store_sales (96) - - -(96) Scan parquet default.store_sales -Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +(37) Scan parquet default.store_sales +Output [4]: [ss_quantity#105, ss_ext_discount_amt#106, ss_net_paid#107, ss_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct +ReadSchema: struct -(97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +(38) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#105, ss_ext_discount_amt#106, ss_net_paid#107, ss_sold_date_sk#108] -(98) Filter [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] -Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) +(39) Filter [codegen id : 1] +Input [4]: [ss_quantity#105, ss_ext_discount_amt#106, ss_net_paid#107, ss_sold_date_sk#108] +Condition : ((isnotnull(ss_quantity#105) AND (ss_quantity#105 >= 81)) AND (ss_quantity#105 <= 100)) -(99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#153] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +(40) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#106, ss_net_paid#107] +Input [4]: [ss_quantity#105, ss_ext_discount_amt#106, ss_net_paid#107, ss_sold_date_sk#108] -(100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#153] +(41) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#106, ss_net_paid#107] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [2]: [sum#155, count#156] -Results [2]: [sum#157, count#158] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#106)), partial_avg(UnscaledValue(ss_net_paid#107))] +Aggregate Attributes [5]: [count#109, sum#110, count#111, sum#112, count#113] +Results [5]: [count#114, sum#115, count#116, sum#117, count#118] -(101) Exchange -Input [2]: [sum#157, count#158] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] +(42) Exchange +Input [5]: [count#114, sum#115, count#116, sum#117, count#118] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] -(102) HashAggregate [codegen id : 2] -Input [2]: [sum#157, count#158] +(43) HashAggregate [codegen id : 2] +Input [5]: [count#114, sum#115, count#116, sum#117, count#118] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] - -Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] -* HashAggregate (109) -+- Exchange (108) - +- * HashAggregate (107) - +- * Project (106) - +- * Filter (105) - +- * ColumnarToRow (104) - +- Scan parquet default.store_sales (103) - - -(103) Scan parquet default.store_sales -Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#106)), avg(UnscaledValue(ss_net_paid#107))] +Aggregate Attributes [3]: [count(1)#120, avg(UnscaledValue(ss_ext_discount_amt#106))#121, avg(UnscaledValue(ss_net_paid#107))#122] +Results [3]: [count(1)#120 AS count(1)#123, cast((avg(UnscaledValue(ss_ext_discount_amt#106))#121 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#124, cast((avg(UnscaledValue(ss_net_paid#107))#122 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#125] -(104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +(44) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#123, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#124, avg(ss_net_paid), avg(ss_net_paid)#125) AS mergedValue#126] +Input [3]: [count(1)#123, avg(ss_ext_discount_amt)#124, avg(ss_net_paid)#125] -(105) Filter [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] -Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) +Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] -(106) Project [codegen id : 1] -Output [1]: [ss_net_paid#163] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] - -(107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#163] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [2]: [sum#165, count#166] -Results [2]: [sum#167, count#168] - -(108) Exchange -Input [2]: [sum#167, count#168] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] - -(109) HashAggregate [codegen id : 2] -Input [2]: [sum#167, count#168] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] +Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt index 1c42d8f2638c6..66ba481fd2045 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt @@ -2,184 +2,79 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #1 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #2 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #2 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #2 + ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #3 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #3 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #3 + ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #4 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #4 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #4 + ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #5 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_sold_date_sk] - Subquery #6 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #6 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid,ss_sold_date_sk] - Subquery #7 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #7 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_sold_date_sk] - Subquery #8 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #8 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_sold_date_sk] - Subquery #9 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #9 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid,ss_sold_date_sk] - Subquery #10 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #10 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_sold_date_sk] - Subquery #11 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #11 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_sold_date_sk] - Subquery #12 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #12 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid,ss_sold_date_sk] - Subquery #13 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #13 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_sold_date_sk] - Subquery #14 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #14 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_sold_date_sk] - Subquery #15 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #15 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #5 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #5 + ReusedSubquery [mergedValue] #5 Filter [r_reason_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index 8736c9861a5ce..8bf63794f25e4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -20,699 +20,284 @@ Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) (4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3] > 62316685) THEN Subquery scalar-subquery#4, [id=#5] ELSE Subquery scalar-subquery#6, [id=#7] END AS bucket1#8, CASE WHEN (Subquery scalar-subquery#9, [id=#10] > 19045798) THEN Subquery scalar-subquery#11, [id=#12] ELSE Subquery scalar-subquery#13, [id=#14] END AS bucket2#15, CASE WHEN (Subquery scalar-subquery#16, [id=#17] > 365541424) THEN Subquery scalar-subquery#18, [id=#19] ELSE Subquery scalar-subquery#20, [id=#21] END AS bucket3#22, CASE WHEN (Subquery scalar-subquery#23, [id=#24] > 216357808) THEN Subquery scalar-subquery#25, [id=#26] ELSE Subquery scalar-subquery#27, [id=#28] END AS bucket4#29, CASE WHEN (Subquery scalar-subquery#30, [id=#31] > 184483884) THEN Subquery scalar-subquery#32, [id=#33] ELSE Subquery scalar-subquery#34, [id=#35] END AS bucket5#36] +Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* HashAggregate (11) -+- Exchange (10) - +- * HashAggregate (9) - +- * Project (8) - +- * Filter (7) - +- * ColumnarToRow (6) - +- Scan parquet default.store_sales (5) +* Project (12) ++- * HashAggregate (11) + +- Exchange (10) + +- * HashAggregate (9) + +- * Project (8) + +- * Filter (7) + +- * ColumnarToRow (6) + +- Scan parquet default.store_sales (5) (5) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct +ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] (7) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) (8) Project [codegen id : 1] -Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_ext_discount_amt#18, ss_net_paid#19] +Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] (9) HashAggregate [codegen id : 1] -Input: [] +Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#39] -Results [1]: [count#40] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] +Aggregate Attributes [5]: [count#21, sum#22, count#23, sum#24, count#25] +Results [5]: [count#26, sum#27, count#28, sum#29, count#30] (10) Exchange -Input [1]: [count#40] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#41] +Input [5]: [count#26, sum#27, count#28, sum#29, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (11) HashAggregate [codegen id : 2] -Input [1]: [count#40] +Input [5]: [count#26, sum#27, count#28, sum#29, count#30] Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#42] -Results [1]: [count(1)#42 AS count(1)#43] - -Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#4, [id=#5] -* HashAggregate (18) -+- Exchange (17) - +- * HashAggregate (16) - +- * Project (15) - +- * Filter (14) - +- * ColumnarToRow (13) - +- Scan parquet default.store_sales (12) - - -(12) Scan parquet default.store_sales -Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] - -(14) Filter [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) - -(15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#45] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] - -(16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#45] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [2]: [sum#47, count#48] -Results [2]: [sum#49, count#50] - -(17) Exchange -Input [2]: [sum#49, count#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] - -(18) HashAggregate [codegen id : 2] -Input [2]: [sum#49, count#50] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] - -Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] -* HashAggregate (25) -+- Exchange (24) - +- * HashAggregate (23) - +- * Project (22) - +- * Filter (21) - +- * ColumnarToRow (20) - +- Scan parquet default.store_sales (19) - - -(19) Scan parquet default.store_sales -Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] - -(21) Filter [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] -Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) - -(22) Project [codegen id : 1] -Output [1]: [ss_net_paid#55] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] - -(23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#55] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [2]: [sum#57, count#58] -Results [2]: [sum#59, count#60] - -(24) Exchange -Input [2]: [sum#59, count#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] - -(25) HashAggregate [codegen id : 2] -Input [2]: [sum#59, count#60] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* HashAggregate (32) -+- Exchange (31) - +- * HashAggregate (30) - +- * Project (29) - +- * Filter (28) - +- * ColumnarToRow (27) - +- Scan parquet default.store_sales (26) - - -(26) Scan parquet default.store_sales -Output [2]: [ss_quantity#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] +Aggregate Attributes [3]: [count(1)#32, avg(UnscaledValue(ss_ext_discount_amt#18))#33, avg(UnscaledValue(ss_net_paid#19))#34] +Results [3]: [count(1)#32 AS count(1)#35, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#33 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#36, cast((avg(UnscaledValue(ss_net_paid#19))#34 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#37] -(27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +(12) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#35, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#36, avg(ss_net_paid), avg(ss_net_paid)#37) AS mergedValue#38] +Input [3]: [count(1)#35, avg(ss_ext_discount_amt)#36, avg(ss_net_paid)#37] -(28) Filter [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] -Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) +Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(29) Project [codegen id : 1] -Output: [] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(30) HashAggregate [codegen id : 1] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#66] -Results [1]: [count#67] +Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] +* Project (20) ++- * HashAggregate (19) + +- Exchange (18) + +- * HashAggregate (17) + +- * Project (16) + +- * Filter (15) + +- * ColumnarToRow (14) + +- Scan parquet default.store_sales (13) -(31) Exchange -Input [1]: [count#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] -(32) HashAggregate [codegen id : 2] -Input [1]: [count#67] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#69] -Results [1]: [count(1)#69 AS count(1)#70] - -Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* HashAggregate (39) -+- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * Filter (35) - +- * ColumnarToRow (34) - +- Scan parquet default.store_sales (33) - - -(33) Scan parquet default.store_sales -Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +(13) Scan parquet default.store_sales +Output [4]: [ss_quantity#39, ss_ext_discount_amt#40, ss_net_paid#41, ss_sold_date_sk#42] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct +ReadSchema: struct -(34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +(14) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#39, ss_ext_discount_amt#40, ss_net_paid#41, ss_sold_date_sk#42] -(35) Filter [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) +(15) Filter [codegen id : 1] +Input [4]: [ss_quantity#39, ss_ext_discount_amt#40, ss_net_paid#41, ss_sold_date_sk#42] +Condition : ((isnotnull(ss_quantity#39) AND (ss_quantity#39 >= 21)) AND (ss_quantity#39 <= 40)) -(36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#72] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +(16) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#40, ss_net_paid#41] +Input [4]: [ss_quantity#39, ss_ext_discount_amt#40, ss_net_paid#41, ss_sold_date_sk#42] -(37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#72] +(17) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#40, ss_net_paid#41] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [2]: [sum#74, count#75] -Results [2]: [sum#76, count#77] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#40)), partial_avg(UnscaledValue(ss_net_paid#41))] +Aggregate Attributes [5]: [count#43, sum#44, count#45, sum#46, count#47] +Results [5]: [count#48, sum#49, count#50, sum#51, count#52] -(38) Exchange -Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +(18) Exchange +Input [5]: [count#48, sum#49, count#50, sum#51, count#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#53] -(39) HashAggregate [codegen id : 2] -Input [2]: [sum#76, count#77] +(19) HashAggregate [codegen id : 2] +Input [5]: [count#48, sum#49, count#50, sum#51, count#52] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] - -Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] -* HashAggregate (46) -+- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * Filter (42) - +- * ColumnarToRow (41) - +- Scan parquet default.store_sales (40) - - -(40) Scan parquet default.store_sales -Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#40)), avg(UnscaledValue(ss_net_paid#41))] +Aggregate Attributes [3]: [count(1)#54, avg(UnscaledValue(ss_ext_discount_amt#40))#55, avg(UnscaledValue(ss_net_paid#41))#56] +Results [3]: [count(1)#54 AS count(1)#57, cast((avg(UnscaledValue(ss_ext_discount_amt#40))#55 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#58, cast((avg(UnscaledValue(ss_net_paid#41))#56 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] -(41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +(20) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#57, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#58, avg(ss_net_paid), avg(ss_net_paid)#59) AS mergedValue#60] +Input [3]: [count(1)#57, avg(ss_ext_discount_amt)#58, avg(ss_net_paid)#59] -(42) Filter [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] -Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) +Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] -(43) Project [codegen id : 1] -Output [1]: [ss_net_paid#82] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] -(44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#82] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] +* Project (28) ++- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * Project (24) + +- * Filter (23) + +- * ColumnarToRow (22) + +- Scan parquet default.store_sales (21) -(45) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] -(46) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] -* HashAggregate (53) -+- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * Filter (49) - +- * ColumnarToRow (48) - +- Scan parquet default.store_sales (47) - - -(47) Scan parquet default.store_sales -Output [2]: [ss_quantity#91, ss_sold_date_sk#92] +(21) Scan parquet default.store_sales +Output [4]: [ss_quantity#61, ss_ext_discount_amt#62, ss_net_paid#63, ss_sold_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct +ReadSchema: struct -(48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +(22) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#61, ss_ext_discount_amt#62, ss_net_paid#63, ss_sold_date_sk#64] -(49) Filter [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] -Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) +(23) Filter [codegen id : 1] +Input [4]: [ss_quantity#61, ss_ext_discount_amt#62, ss_net_paid#63, ss_sold_date_sk#64] +Condition : ((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 41)) AND (ss_quantity#61 <= 60)) -(50) Project [codegen id : 1] -Output: [] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +(24) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#62, ss_net_paid#63] +Input [4]: [ss_quantity#61, ss_ext_discount_amt#62, ss_net_paid#63, ss_sold_date_sk#64] -(51) HashAggregate [codegen id : 1] -Input: [] +(25) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#62, ss_net_paid#63] Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#93] -Results [1]: [count#94] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#62)), partial_avg(UnscaledValue(ss_net_paid#63))] +Aggregate Attributes [5]: [count#65, sum#66, count#67, sum#68, count#69] +Results [5]: [count#70, sum#71, count#72, sum#73, count#74] -(52) Exchange -Input [1]: [count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +(26) Exchange +Input [5]: [count#70, sum#71, count#72, sum#73, count#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] -(53) HashAggregate [codegen id : 2] -Input [1]: [count#94] +(27) HashAggregate [codegen id : 2] +Input [5]: [count#70, sum#71, count#72, sum#73, count#74] Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#96] -Results [1]: [count(1)#96 AS count(1)#97] - -Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] -* HashAggregate (60) -+- Exchange (59) - +- * HashAggregate (58) - +- * Project (57) - +- * Filter (56) - +- * ColumnarToRow (55) - +- Scan parquet default.store_sales (54) - - -(54) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#62)), avg(UnscaledValue(ss_net_paid#63))] +Aggregate Attributes [3]: [count(1)#76, avg(UnscaledValue(ss_ext_discount_amt#62))#77, avg(UnscaledValue(ss_net_paid#63))#78] +Results [3]: [count(1)#76 AS count(1)#79, cast((avg(UnscaledValue(ss_ext_discount_amt#62))#77 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80, cast((avg(UnscaledValue(ss_net_paid#63))#78 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#81] -(55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +(28) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#79, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#80, avg(ss_net_paid), avg(ss_net_paid)#81) AS mergedValue#82] +Input [3]: [count(1)#79, avg(ss_ext_discount_amt)#80, avg(ss_net_paid)#81] -(56) Filter [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] -Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) +Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] -(57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#99] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] -(58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#99] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [2]: [sum#101, count#102] -Results [2]: [sum#103, count#104] +Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* Project (36) ++- * HashAggregate (35) + +- Exchange (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * ColumnarToRow (30) + +- Scan parquet default.store_sales (29) -(59) Exchange -Input [2]: [sum#103, count#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] -(60) HashAggregate [codegen id : 2] -Input [2]: [sum#103, count#104] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] - -Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * Filter (63) - +- * ColumnarToRow (62) - +- Scan parquet default.store_sales (61) - - -(61) Scan parquet default.store_sales -Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] - -(63) Filter [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] -Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) - -(64) Project [codegen id : 1] -Output [1]: [ss_net_paid#109] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] - -(65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#109] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [2]: [sum#111, count#112] -Results [2]: [sum#113, count#114] - -(66) Exchange -Input [2]: [sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] - -(67) HashAggregate [codegen id : 2] -Input [2]: [sum#113, count#114] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] -* HashAggregate (74) -+- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * Filter (70) - +- * ColumnarToRow (69) - +- Scan parquet default.store_sales (68) - - -(68) Scan parquet default.store_sales -Output [2]: [ss_quantity#118, ss_sold_date_sk#119] +(29) Scan parquet default.store_sales +Output [4]: [ss_quantity#83, ss_ext_discount_amt#84, ss_net_paid#85, ss_sold_date_sk#86] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct +ReadSchema: struct -(69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +(30) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#83, ss_ext_discount_amt#84, ss_net_paid#85, ss_sold_date_sk#86] -(70) Filter [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] -Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) +(31) Filter [codegen id : 1] +Input [4]: [ss_quantity#83, ss_ext_discount_amt#84, ss_net_paid#85, ss_sold_date_sk#86] +Condition : ((isnotnull(ss_quantity#83) AND (ss_quantity#83 >= 61)) AND (ss_quantity#83 <= 80)) -(71) Project [codegen id : 1] -Output: [] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +(32) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#84, ss_net_paid#85] +Input [4]: [ss_quantity#83, ss_ext_discount_amt#84, ss_net_paid#85, ss_sold_date_sk#86] -(72) HashAggregate [codegen id : 1] -Input: [] +(33) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#84, ss_net_paid#85] Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#120] -Results [1]: [count#121] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#84)), partial_avg(UnscaledValue(ss_net_paid#85))] +Aggregate Attributes [5]: [count#87, sum#88, count#89, sum#90, count#91] +Results [5]: [count#92, sum#93, count#94, sum#95, count#96] -(73) Exchange -Input [1]: [count#121] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] +(34) Exchange +Input [5]: [count#92, sum#93, count#94, sum#95, count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] -(74) HashAggregate [codegen id : 2] -Input [1]: [count#121] +(35) HashAggregate [codegen id : 2] +Input [5]: [count#92, sum#93, count#94, sum#95, count#96] Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#123] -Results [1]: [count(1)#123 AS count(1)#124] - -Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] -* HashAggregate (81) -+- Exchange (80) - +- * HashAggregate (79) - +- * Project (78) - +- * Filter (77) - +- * ColumnarToRow (76) - +- Scan parquet default.store_sales (75) - - -(75) Scan parquet default.store_sales -Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#84)), avg(UnscaledValue(ss_net_paid#85))] +Aggregate Attributes [3]: [count(1)#98, avg(UnscaledValue(ss_ext_discount_amt#84))#99, avg(UnscaledValue(ss_net_paid#85))#100] +Results [3]: [count(1)#98 AS count(1)#101, cast((avg(UnscaledValue(ss_ext_discount_amt#84))#99 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#102, cast((avg(UnscaledValue(ss_net_paid#85))#100 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#103] -(76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +(36) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#101, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#102, avg(ss_net_paid), avg(ss_net_paid)#103) AS mergedValue#104] +Input [3]: [count(1)#101, avg(ss_ext_discount_amt)#102, avg(ss_net_paid)#103] -(77) Filter [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] -Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) +Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -(78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#126] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -(79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#126] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [2]: [sum#128, count#129] -Results [2]: [sum#130, count#131] +Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] +* Project (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * Filter (39) + +- * ColumnarToRow (38) + +- Scan parquet default.store_sales (37) -(80) Exchange -Input [2]: [sum#130, count#131] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] -(81) HashAggregate [codegen id : 2] -Input [2]: [sum#130, count#131] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] - -Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* HashAggregate (88) -+- Exchange (87) - +- * HashAggregate (86) - +- * Project (85) - +- * Filter (84) - +- * ColumnarToRow (83) - +- Scan parquet default.store_sales (82) - - -(82) Scan parquet default.store_sales -Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] - -(84) Filter [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] -Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) - -(85) Project [codegen id : 1] -Output [1]: [ss_net_paid#136] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] - -(86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#136] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [2]: [sum#138, count#139] -Results [2]: [sum#140, count#141] - -(87) Exchange -Input [2]: [sum#140, count#141] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] - -(88) HashAggregate [codegen id : 2] -Input [2]: [sum#140, count#141] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* HashAggregate (95) -+- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * Filter (91) - +- * ColumnarToRow (90) - +- Scan parquet default.store_sales (89) - - -(89) Scan parquet default.store_sales -Output [2]: [ss_quantity#145, ss_sold_date_sk#146] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] - -(91) Filter [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] -Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) - -(92) Project [codegen id : 1] -Output: [] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] - -(93) HashAggregate [codegen id : 1] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#147] -Results [1]: [count#148] - -(94) Exchange -Input [1]: [count#148] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] - -(95) HashAggregate [codegen id : 2] -Input [1]: [count#148] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#150] -Results [1]: [count(1)#150 AS count(1)#151] - -Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] -* HashAggregate (102) -+- Exchange (101) - +- * HashAggregate (100) - +- * Project (99) - +- * Filter (98) - +- * ColumnarToRow (97) - +- Scan parquet default.store_sales (96) - - -(96) Scan parquet default.store_sales -Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +(37) Scan parquet default.store_sales +Output [4]: [ss_quantity#105, ss_ext_discount_amt#106, ss_net_paid#107, ss_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct +ReadSchema: struct -(97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +(38) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#105, ss_ext_discount_amt#106, ss_net_paid#107, ss_sold_date_sk#108] -(98) Filter [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] -Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) +(39) Filter [codegen id : 1] +Input [4]: [ss_quantity#105, ss_ext_discount_amt#106, ss_net_paid#107, ss_sold_date_sk#108] +Condition : ((isnotnull(ss_quantity#105) AND (ss_quantity#105 >= 81)) AND (ss_quantity#105 <= 100)) -(99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#153] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +(40) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#106, ss_net_paid#107] +Input [4]: [ss_quantity#105, ss_ext_discount_amt#106, ss_net_paid#107, ss_sold_date_sk#108] -(100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#153] +(41) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#106, ss_net_paid#107] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [2]: [sum#155, count#156] -Results [2]: [sum#157, count#158] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#106)), partial_avg(UnscaledValue(ss_net_paid#107))] +Aggregate Attributes [5]: [count#109, sum#110, count#111, sum#112, count#113] +Results [5]: [count#114, sum#115, count#116, sum#117, count#118] -(101) Exchange -Input [2]: [sum#157, count#158] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] +(42) Exchange +Input [5]: [count#114, sum#115, count#116, sum#117, count#118] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] -(102) HashAggregate [codegen id : 2] -Input [2]: [sum#157, count#158] +(43) HashAggregate [codegen id : 2] +Input [5]: [count#114, sum#115, count#116, sum#117, count#118] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] - -Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] -* HashAggregate (109) -+- Exchange (108) - +- * HashAggregate (107) - +- * Project (106) - +- * Filter (105) - +- * ColumnarToRow (104) - +- Scan parquet default.store_sales (103) - - -(103) Scan parquet default.store_sales -Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#106)), avg(UnscaledValue(ss_net_paid#107))] +Aggregate Attributes [3]: [count(1)#120, avg(UnscaledValue(ss_ext_discount_amt#106))#121, avg(UnscaledValue(ss_net_paid#107))#122] +Results [3]: [count(1)#120 AS count(1)#123, cast((avg(UnscaledValue(ss_ext_discount_amt#106))#121 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#124, cast((avg(UnscaledValue(ss_net_paid#107))#122 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#125] -(104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +(44) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#123, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#124, avg(ss_net_paid), avg(ss_net_paid)#125) AS mergedValue#126] +Input [3]: [count(1)#123, avg(ss_ext_discount_amt)#124, avg(ss_net_paid)#125] -(105) Filter [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] -Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) +Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] -(106) Project [codegen id : 1] -Output [1]: [ss_net_paid#163] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] - -(107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#163] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [2]: [sum#165, count#166] -Results [2]: [sum#167, count#168] - -(108) Exchange -Input [2]: [sum#167, count#168] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] - -(109) HashAggregate [codegen id : 2] -Input [2]: [sum#167, count#168] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] +Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index 1c42d8f2638c6..66ba481fd2045 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -2,184 +2,79 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #1 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #2 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #2 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #2 + ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #3 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #3 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #3 + ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #4 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #4 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #4 + ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #5 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_sold_date_sk] - Subquery #6 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #6 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid,ss_sold_date_sk] - Subquery #7 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #7 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_sold_date_sk] - Subquery #8 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #8 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_sold_date_sk] - Subquery #9 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #9 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid,ss_sold_date_sk] - Subquery #10 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #10 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_sold_date_sk] - Subquery #11 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #11 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_sold_date_sk] - Subquery #12 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #12 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid,ss_sold_date_sk] - Subquery #13 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #13 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_sold_date_sk] - Subquery #14 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #14 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_sold_date_sk] - Subquery #15 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #15 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #5 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #5 + ReusedSubquery [mergedValue] #5 Filter [r_reason_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala index 6065f2321091b..884d1338ad1eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala @@ -264,28 +264,25 @@ class InjectRuntimeFilterSuite extends QueryTest with SQLTestUtils with SharedSp } } - def getNumBloomFilters(plan: LogicalPlan): Integer = { - val numBloomFilterAggs = plan.collect { - case Filter(condition, _) => condition.collect { - case subquery: org.apache.spark.sql.catalyst.expressions.ScalarSubquery - => subquery.plan.collect { - case Aggregate(_, aggregateExpressions, _) => - aggregateExpressions.map { - case Alias(AggregateExpression(bfAgg : BloomFilterAggregate, _, _, _, _), - _) => - assert(bfAgg.estimatedNumItemsExpression.isInstanceOf[Literal]) - assert(bfAgg.numBitsExpression.isInstanceOf[Literal]) - 1 - }.sum + // `MergeScalarSubqueries` can duplicate subqueries in the optimized plan, but the subqueries will + // be reused in the physical plan. + def getNumBloomFilters(plan: LogicalPlan, scalarSubqueryCTEMultiplicator: Int = 1): Integer = { + print(plan) + val numBloomFilterAggs = plan.collectWithSubqueries { + case Aggregate(_, aggregateExpressions, _) => + aggregateExpressions.collect { + case Alias(AggregateExpression(bfAgg: BloomFilterAggregate, _, _, _, _), _) => + assert(bfAgg.estimatedNumItemsExpression.isInstanceOf[Literal]) + assert(bfAgg.numBitsExpression.isInstanceOf[Literal]) + 1 }.sum - }.sum }.sum val numMightContains = plan.collect { case Filter(condition, _) => condition.collect { case BloomFilterMightContain(_, _) => 1 }.sum }.sum - assert(numBloomFilterAggs == numMightContains) + assert(numBloomFilterAggs == numMightContains * scalarSubqueryCTEMultiplicator) numMightContains } @@ -389,7 +386,7 @@ class InjectRuntimeFilterSuite extends QueryTest with SQLTestUtils with SharedSp planEnabled = sql(query).queryExecution.optimizedPlan checkAnswer(sql(query), expectedAnswer) } - assert(getNumBloomFilters(planEnabled) == getNumBloomFilters(planDisabled) + 2) + assert(getNumBloomFilters(planEnabled, 2) == getNumBloomFilters(planDisabled) + 2) } } @@ -417,10 +414,10 @@ class InjectRuntimeFilterSuite extends QueryTest with SQLTestUtils with SharedSp checkAnswer(sql(query), expectedAnswer) } if (numFilterThreshold < 3) { - assert(getNumBloomFilters(planEnabled) == getNumBloomFilters(planDisabled) - + numFilterThreshold) + assert(getNumBloomFilters(planEnabled, numFilterThreshold) == + getNumBloomFilters(planDisabled) + numFilterThreshold) } else { - assert(getNumBloomFilters(planEnabled) == getNumBloomFilters(planDisabled) + 2) + assert(getNumBloomFilters(planEnabled, 2) == getNumBloomFilters(planDisabled) + 2) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 92c373a33fb24..221663c61e18d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2018,4 +2018,171 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark }.getMessage.contains("Correlated column is not allowed in predicate")) } } + + test("Merge non-correlated scalar subqueries") { + Seq(false, true).foreach { enableAQE => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString) { + val df = sql( + """ + |SELECT + | (SELECT avg(key) FROM testData), + | (SELECT sum(key) FROM testData), + | (SELECT count(distinct key) FROM testData) + """.stripMargin) + + checkAnswer(df, Row(50.5, 5050, 100) :: Nil) + + val plan = df.queryExecution.executedPlan + val subqueryIds = collectWithSubqueries(plan) { case s: SubqueryExec => s.id } + val reusedSubqueryIds = collectWithSubqueries(plan) { + case rs: ReusedSubqueryExec => rs.child.id + } + + assert(subqueryIds.size == 1, "Missing or unexpected SubqueryExec in the plan") + assert(reusedSubqueryIds.size == 2, + "Missing or unexpected reused ReusedSubqueryExec in the plan") + } + } + } + + test("Merge non-correlated scalar subqueries in a subquery") { + Seq(false, true).foreach { enableAQE => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString) { + val df = sql( + """ + |SELECT ( + | SELECT + | SUM( + | (SELECT avg(key) FROM testData) + + | (SELECT sum(key) FROM testData) + + | (SELECT count(distinct key) FROM testData)) + | FROM testData + |) + """.stripMargin) + + checkAnswer(df, Row(520050.0) :: Nil) + + val plan = df.queryExecution.executedPlan + val subqueryIds = collectWithSubqueries(plan) { case s: SubqueryExec => s.id } + val reusedSubqueryIds = collectWithSubqueries(plan) { + case rs: ReusedSubqueryExec => rs.child.id + } + + if (enableAQE) { + assert(subqueryIds.size == 3, "Missing or unexpected SubqueryExec in the plan") + assert(reusedSubqueryIds.size == 4, + "Missing or unexpected reused ReusedSubqueryExec in the plan") + } else { + assert(subqueryIds.size == 2, "Missing or unexpected SubqueryExec in the plan") + assert(reusedSubqueryIds.size == 5, + "Missing or unexpected reused ReusedSubqueryExec in the plan") + } + } + } + } + + test("Merge non-correlated scalar subqueries from different levels") { + Seq(false, true).foreach { enableAQE => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString) { + val df = sql( + """ + |SELECT + | (SELECT avg(key) FROM testData), + | ( + | SELECT + | SUM( + | (SELECT sum(key) FROM testData) + | ) + | FROM testData + | ) + """.stripMargin) + + checkAnswer(df, Row(50.5, 505000) :: Nil) + + val plan = df.queryExecution.executedPlan + val subqueryIds = collectWithSubqueries(plan) { case s: SubqueryExec => s.id } + val reusedSubqueryIds = collectWithSubqueries(plan) { + case rs: ReusedSubqueryExec => rs.child.id + } + + assert(subqueryIds.size == 2, "Missing or unexpected SubqueryExec in the plan") + assert(reusedSubqueryIds.size == 2, + "Missing or unexpected reused ReusedSubqueryExec in the plan") + } + } + } + + test("Merge non-correlated scalar subqueries from different parent plans") { + Seq(false, true).foreach { enableAQE => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString) { + val df = sql( + """ + |SELECT + | ( + | SELECT + | SUM( + | (SELECT avg(key) FROM testData) + | ) + | FROM testData + | ), + | ( + | SELECT + | SUM( + | (SELECT sum(key) FROM testData) + | ) + | FROM testData + | ) + """.stripMargin) + + checkAnswer(df, Row(5050.0, 505000) :: Nil) + + val plan = df.queryExecution.executedPlan + val subqueryIds = collectWithSubqueries(plan) { case s: SubqueryExec => s.id } + val reusedSubqueryIds = collectWithSubqueries(plan) { + case rs: ReusedSubqueryExec => rs.child.id + } + + if (enableAQE) { + assert(subqueryIds.size == 3, "Missing or unexpected SubqueryExec in the plan") + assert(reusedSubqueryIds.size == 3, + "Missing or unexpected reused ReusedSubqueryExec in the plan") + } else { + assert(subqueryIds.size == 2, "Missing or unexpected SubqueryExec in the plan") + assert(reusedSubqueryIds.size == 4, + "Missing or unexpected reused ReusedSubqueryExec in the plan") + } + } + } + } + + test("Merge non-correlated scalar subqueries with conflicting names") { + Seq(false, true).foreach { enableAQE => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString) { + val df = sql( + """ + |SELECT + | (SELECT avg(key) AS key FROM testData), + | (SELECT sum(key) AS key FROM testData), + | (SELECT count(distinct key) AS key FROM testData) + """.stripMargin) + + checkAnswer(df, Row(50.5, 5050, 100) :: Nil) + + val plan = df.queryExecution.executedPlan + val subqueryIds = collectWithSubqueries(plan) { case s: SubqueryExec => s.id } + val reusedSubqueryIds = collectWithSubqueries(plan) { + case rs: ReusedSubqueryExec => rs.child.id + } + + assert(subqueryIds.size == 1, "Missing or unexpected SubqueryExec in the plan") + assert(reusedSubqueryIds.size == 2, + "Missing or unexpected reused ReusedSubqueryExec in the plan") + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala index 66f01a0709110..743ec41dbe7cd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.TPCDSQuerySuite import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, Final} -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Generate, Join, LocalRelation, LogicalPlan, Range, Sample, Union, Window} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Generate, Join, LocalRelation, LogicalPlan, Range, Sample, Union, Window, WithCTE} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} @@ -108,7 +108,11 @@ class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite with DisableAdaptiv // logical = Project(Filter(Scan A)) // physical = ProjectExec(ScanExec A) // we only check that leaf modes match between logical and physical plan. - val logicalLeaves = getLogicalPlan(actualPlan).collectLeaves() + val logicalPlan = getLogicalPlan(actualPlan) match { + case w: WithCTE => w.plan + case o => o + } + val logicalLeaves = logicalPlan.collectLeaves() val physicalLeaves = plan.collectLeaves() assert(logicalLeaves.length == 1) assert(physicalLeaves.length == 1)