From 66f768310a5d01022d717b47bd6a21e78155f2c7 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Fri, 10 Apr 2026 14:16:44 +0200 Subject: [PATCH] [SPARK-40193][SQL] Merge subplans with different filter conditions ### What changes were proposed in this pull request? `PlanMerger` is extended to merge non-correlated non-grouping aggregate subplans that differ only in their `WHERE` filter conditions. Filter merging follows the same recursive plan-matching logic as the rest of `PlanMerger` and handles three cases: - `(np: Filter, cp: Filter)` with different conditions: both conditions are aliased as boolean attributes in a `Project`, a merged `Filter(OR(f0, f1))` is introduced, and the aliases are propagated up to the enclosing `Aggregate` where each side's expressions receive a `FILTER (WHERE ...)` clause. - `(np: Filter, cp)` or `(np, cp: Filter)`: only one side has a filter; the condition is exposed as a `Project` attribute and propagated up so only that side's aggregate expressions receive a `FILTER` clause. - Equal filter conditions pass through unchanged. When plans also differ in intermediate `Project` expressions above a `Filter`, those expressions are wrapped with `If(filterAttr, expr, null)` to avoid computing them for rows that do not match that side's filter condition. Plain attribute references are never wrapped since reading a column value is free. **Example** ``` // Input plans Aggregate [sum(a) AS sum_a] Aggregate [max(d) AS max_d] +- Filter (a < 1) +- Project [udf(a) AS d] +- Scan t +- Filter (a > 1) +- Scan t // Merged plan Aggregate [sum(a) FILTER f0 AS sum_a, max(d0) FILTER f1 AS max_d] +- Project [a, If(f1, udf(a), null) AS d0, f0, f1] +- Filter (f0 OR f1) +- Project [a, (a < 1) AS f0, (a > 1) AS f1] +- Scan t ``` **Benefit**: a single scan of `t` computes both aggregates, which is typically cheaper than two separate scans. The `If` wrapping ensures `udf(a)` is only evaluated for rows that match `a > 1`. **Drawback** (symmetric case only): the merged `Filter(f0 OR f1)` is less selective than each individual filter, which may reduce IO pruning such as partition or file skipping. On heavily partitioned or file-pruned tables the extra IO can outweigh the scan-deduplication benefit. The asymmetric case (`(np: Filter, cp)`) is always beneficial because the unfiltered side would have read all the data anyway. **Configs**: - `spark.sql.planMerge.filterPropagation.enabled` (internal, default `true`): master switch; disabling it turns off all filter-based merging. - `spark.sql.planMerge.symmetricFilterPropagation.enabled` (default `true`): controls the symmetric `(Filter, Filter)` case specifically, so users on IO-pruning-sensitive workloads can disable only that path while keeping the always-beneficial asymmetric merging. `MergeResult.outputMap` is changed from `AttributeMap[Attribute]` to `AttributeMap[Int]`, mapping each input plan attribute to its positional index in the merged output. Positional indices remain stable across subsequent `PlanMerger.merge` calls (outputs are only ever appended), whereas retained `Attribute` values can become stale when filter merging replaces expressions with new aliases. This also simplifies the two call sites in `MergeSubplans`. ### Why are the changes needed? Computing aggregates over the same table with different `WHERE` clauses is a common analytical pattern (e.g. conditional sums or counts for different predicates). Without this change each subquery forces a separate full scan; merging them reduces scan count and overall query cost. ### Does this PR introduce _any_ user-facing change? Yes. A new config `spark.sql.planMerge.symmetricFilterPropagation.enabled` (default `true`) is added. The optimization is otherwise transparent: queries produce the same results, and both configs can be set to `false` to restore the previous behavior. ### How was this patch tested? New unit tests in `MergeSubplansSuite` and new end-to-end tests in `PlanMergeSuite` covering the basic two-subplan cases, three-subplan merging, disabled configs, grouping aggregates (not merged), asymmetric filters, stacked filters, and reversed filter ordering. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Sonnet 4.6 (Anthropic) --- .../catalyst/optimizer/MergeSubplans.scala | 8 +- .../sql/catalyst/optimizer/PlanMerger.scala | 469 ++++++++++--- .../apache/spark/sql/internal/SQLConf.scala | 25 + .../optimizer/MergeSubplansSuite.scala | 654 +++++++++++++++++- .../q59.sf100/explain.txt | 161 ++--- .../q59.sf100/simplified.txt | 36 +- .../approved-plans-v1_4/q2.sf100/explain.txt | 165 ++--- .../q2.sf100/simplified.txt | 34 +- .../approved-plans-v1_4/q28.sf100/explain.txt | 566 +++++---------- .../q28.sf100/simplified.txt | 158 ++--- .../approved-plans-v1_4/q28/explain.txt | 566 +++++---------- .../approved-plans-v1_4/q28/simplified.txt | 158 ++--- .../approved-plans-v1_4/q59.sf100/explain.txt | 161 ++--- .../q59.sf100/simplified.txt | 36 +- .../approved-plans-v1_4/q9.sf100/explain.txt | 260 +------ .../q9.sf100/simplified.txt | 80 +-- .../approved-plans-v1_4/q9/explain.txt | 260 +------ .../approved-plans-v1_4/q9/simplified.txt | 80 +-- .../org/apache/spark/sql/PlanMergeSuite.scala | 56 ++ .../apache/spark/sql/PlanStabilitySuite.scala | 15 +- 20 files changed, 1919 insertions(+), 2029 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeSubplans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeSubplans.scala index 94b7da8da21f3..5c6b531119ece 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeSubplans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeSubplans.scala @@ -235,9 +235,7 @@ object MergeSubplans extends Rule[LogicalPlan] { levelFromSubqueries = levelFromSubqueries.max(level + 1) - val mergedOutput = mergeResult.outputMap(planWithReferences.output.head) - val outputIndex = - mergeResult.mergedPlan.plan.output.indexWhere(_.exprId == mergedOutput.exprId) + val outputIndex = mergeResult.outputMap(planWithReferences.output.head) ScalarSubqueryReference( level, mergeResult.mergedPlanIndex, @@ -259,9 +257,7 @@ object MergeSubplans extends Rule[LogicalPlan] { val mergeResult = getPlanMerger(planMergers, level).merge(aggregateWithReferences, false) - val mergedOutput = aggregateWithReferences.output.map(mergeResult.outputMap) - val outputIndices = - mergedOutput.map(a => mergeResult.mergedPlan.plan.output.indexWhere(_.exprId == a.exprId)) + val outputIndices = aggregateWithReferences.output.map(mergeResult.outputMap) val aggregateReference = NonGroupingAggregateReference( level, mergeResult.mergedPlanIndex, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PlanMerger.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PlanMerger.scala index 1623166e0a657..c76cec3a26781 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PlanMerger.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PlanMerger.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.catalyst.optimizer -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, If, Literal, NamedExpression, Or} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.trees.TreeNodeTag +import org.apache.spark.sql.internal.SQLConf /** * Result of attempting to merge a plan via [[PlanMerger.merge]]. @@ -31,14 +33,14 @@ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, Log * - A newly merged plan combining the input with a cached plan * - The original input plan (if no merge was possible) * @param mergedPlanIndex The index of this plan in the PlanMerger's cache. - * @param outputMap Maps attributes from the input plan to corresponding attributes in - * `mergedPlan`. Used to rewrite expressions referencing the original plan - * to reference the merged plan instead. + * @param outputMap Maps attributes of the input plan to their positional index in + * `mergedPlan.plan.output`. The index remains stable across subsequent + * [[PlanMerger.merge]] calls because outputs are only ever appended. */ case class MergeResult( mergedPlan: MergedPlan, mergedPlanIndex: Int, - outputMap: AttributeMap[Attribute]) + outputMap: AttributeMap[Int]) /** * Represents a plan in the PlanMerger's cache. @@ -50,6 +52,19 @@ case class MergeResult( */ case class MergedPlan(plan: LogicalPlan, merged: Boolean) +object PlanMerger { + // Marker tag placed on Filter nodes that were produced by filter propagation. Its presence + // signals that the Filter's condition is already an OR of propagated filter attributes and + // its child Project already contains the corresponding aliases, so a subsequent merge only + // needs to add one new alias for the incoming plan rather than wrapping both sides again. + val MERGED_FILTER_TAG: TreeNodeTag[Unit] = TreeNodeTag("mergedFilter") + + // Global counter for generating unique names for propagated filter attributes across all + // PlanMerger instances. + private[optimizer] val curId = new java.util.concurrent.atomic.AtomicLong() + private[optimizer] def newId: Long = curId.getAndIncrement() +} + /** * A stateful utility for merging identical or similar logical plans to enable query plan reuse. * @@ -67,6 +82,31 @@ case class MergedPlan(plan: LogicalPlan, merged: Boolean) * - [[Filter]]: Requires identical filter conditions * - [[Join]]: Requires identical join type, hints, and conditions * + * When `filterPropagationEnabled` is true, non-grouping [[Aggregate]]s over the same base plan + * with different [[Filter]] conditions can also be merged. The filter conditions are exposed as + * boolean [[Project]] attributes and consumed at the [[Aggregate]] as FILTER clauses. + * When both sides carry a [[Filter]] (the symmetric case), merging broadens the scan to + * OR(f1, f2), which may reduce IO pruning. This path is separately gated by + * `symmetricFilterPropagationEnabled`. + * When plans also differ in intermediate [[Project]] expressions, those are wrapped with + * `If(filterAttr, expr, null)` to avoid computing the expression for rows that do not + * match that side's filter condition. + * + * {{{ + * // Input plans + * Aggregate [sum(a) AS sum_a] Aggregate [max(d) AS max_d] + * +- Filter (a < 1) +- Project [udf(a) AS d] + * +- Scan t +- Filter (a > 1) + * +- Scan t + * + * // Merged plan + * Aggregate [sum(a) FILTER f0 AS sum_a, max(d0) FILTER f1 AS max_d] + * +- Project [a, If(f1, udf(a), null) AS d0, f0, f1] + * +- Filter (f0 OR f1) [MERGED_FILTER_TAG] + * +- Project [a, (a < 1) AS f0, (a > 1) AS f1] + * +- Scan t + * }}} + * * @example * {{{ * val merger = PlanMerger() @@ -76,8 +116,12 @@ case class MergedPlan(plan: LogicalPlan, merged: Boolean) * // result2.outputMap maps plan2's attributes to the merged plan's attributes * }}} */ -class PlanMerger { - val cache = ArrayBuffer.empty[MergedPlan] +class PlanMerger( + filterPropagationEnabled: Boolean = + SQLConf.get.getConf(SQLConf.PLAN_MERGE_FILTER_PROPAGATION_ENABLED), + symmetricFilterPropagationEnabled: Boolean = + SQLConf.get.getConf(SQLConf.PLAN_MERGE_SYMMETRIC_FILTER_PROPAGATION_ENABLED)) { + val cache = mutable.ArrayBuffer.empty[MergedPlan] /** * Attempts to merge the given plan with cached plans, or adds it to the cache. @@ -97,19 +141,23 @@ class PlanMerger { def merge(plan: LogicalPlan, subqueryPlan: Boolean): MergeResult = { cache.zipWithIndex.collectFirst(Function.unlift { case (mp, i) => - checkIdenticalPlans(plan, mp.plan).map { outputMap => + checkIdenticalPlans(plan, mp.plan).map { _ => // Identical subquery expression plans are not marked as `merged` as the // `ReusedSubqueryExec` rule can handle them without extracting the plans to CTEs. // But, when a non-subquery subplan is identical to a cached plan we need to mark the plan // `merged` and so extract it to a CTE later. - val newMergePlan = MergedPlan(mp.plan, cache(i).merged || !subqueryPlan) - cache(i) = newMergePlan - MergeResult(newMergePlan, i, outputMap) + val newMergedPlan = MergedPlan(mp.plan, cache(i).merged || !subqueryPlan) + cache(i) = newMergedPlan + val outputMap = AttributeMap(plan.output.zipWithIndex) + MergeResult(newMergedPlan, i, outputMap) }.orElse { - tryMergePlans(plan, mp.plan).map { - case (mergedPlan, outputMap) => + tryMergePlans(plan, mp.plan, false).collect { + case TryMergeResult(mergedPlan, npMapping, _, None, None) => val newMergePlan = MergedPlan(mergedPlan, true) cache(i) = newMergePlan + val outputMap = AttributeMap(npMapping.iterator.map { case (origAttr, mergedAttr) => + origAttr -> mergedPlan.output.indexWhere(_.exprId == mergedAttr.exprId) + }.toSeq) MergeResult(newMergePlan, i, outputMap) } } @@ -117,7 +165,7 @@ class PlanMerger { }).getOrElse { val newMergePlan = MergedPlan(plan, false) cache += newMergePlan - val outputMap = AttributeMap(plan.output.map(a => a -> a)) + val outputMap = AttributeMap(plan.output.zipWithIndex) MergeResult(newMergePlan, cache.length - 1, outputMap) } } @@ -141,6 +189,35 @@ class PlanMerger { } } + /** + * Result of a successful [[tryMergePlans]] call. + * + * @param mergedPlan The combined logical plan. + * @param newPlanMapping Mapping from attributes in the new plan to the corresponding + * attributes in the merged plan. Used by parent nodes to remap + * new-plan-side expressions. + * @param cachedPlanMapping Mapping from original cached-plan attributes to their new alias + * attributes when a cached expression was wrapped with an `If`. Used by + * parent nodes to remap cached-plan-side expressions that would + * otherwise reference stale attributes after wrapping. Empty when no + * cached expressions were wrapped. + * @param newPlanFilter A boolean [[Attribute]] in the merged plan that encodes the filter + * condition from the new plan's side, to be applied as an aggregate + * `FILTER (WHERE ...)` clause when the propagation reaches an enclosing + * [[Aggregate]] node. The boolean component is `true` if the attribute was + * freshly aliased and must be appended to enclosing [[Project]] nodes, or + * `false` if it was reused from an existing alias already present in the + * merged plan. `None` when no differing filter was propagated. + * @param cachedPlanFilter Like `newPlanFilter` but for the cached plan's side. Always a freshly + * created alias when present, so no `isNew` flag is needed. + */ + case class TryMergeResult( + mergedPlan: LogicalPlan, + newPlanMapping: AttributeMap[Attribute], + cachedPlanMapping: AttributeMap[Attribute] = AttributeMap.empty, + newPlanFilter: Option[(Attribute, Boolean)] = None, + cachedPlanFilter: Option[Attribute] = None) + /** * Recursively attempts to merge two plans by traversing their tree structures. * @@ -157,83 +234,219 @@ class PlanMerger { * * @param newPlan The plan to merge into the cached plan. * @param cachedPlan The cached plan to merge with. - * @return Some((mergedPlan, outputMap)) if merge succeeds, where: - * - mergedPlan is the combined plan - * - outputMap maps newPlan's attributes to mergedPlan's attributes - * Returns None if plans cannot be merged. + * @return Some([[TryMergeResult]]) if merge succeeds, None if plans cannot be merged. */ private def tryMergePlans( newPlan: LogicalPlan, - cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = { - checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse( + cachedPlan: LogicalPlan, + filterPropagationSupported: Boolean): Option[TryMergeResult] = { + checkIdenticalPlans(newPlan, cachedPlan).map(TryMergeResult(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 + tryMergePlans(np.child, cp.child, filterPropagationSupported).map { + case TryMergeResult(mergedChild, npMapping, cpMapping, npFilter, cpFilter) => + val (mergedProjectList, newNPMapping, newCPMapping) = + mergeNamedExpressions(np.projectList, cp.projectList, npMapping, cpMapping, + npFilter, cpFilter) + TryMergeResult(Project(mergedProjectList, mergedChild), newNPMapping, newCPMapping, + npFilter, cpFilter) } 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 + tryMergePlans(np, cp.child, filterPropagationSupported).map { + case TryMergeResult(mergedChild, npMapping, cpMapping, npFilter, cpFilter) => + val (mergedProjectList, newNPMapping, newCPMapping) = + mergeNamedExpressions(np.output, cp.projectList, npMapping, cpMapping, npFilter, + cpFilter) + TryMergeResult(Project(mergedProjectList, mergedChild), newNPMapping, newCPMapping, + npFilter, cpFilter) } 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 + tryMergePlans(np.child, cp, filterPropagationSupported).map { + case TryMergeResult(mergedChild, npMapping, cpMapping, npFilter, cpFilter) => + val (mergedProjectList, newNPMapping, newCPMapping) = + mergeNamedExpressions(np.projectList, cp.output, npMapping, cpMapping, npFilter, + cpFilter) + TryMergeResult(Project(mergedProjectList, mergedChild), newNPMapping, newCPMapping, + npFilter, cpFilter) } + 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 - } - } + // Filter propagation into the aggregate is only safe when there is no grouping. + val childFilterPropagationSupported = filterPropagationEnabled && + np.groupingExpressions.isEmpty && cp.groupingExpressions.isEmpty + tryMergePlans(np.child, cp.child, childFilterPropagationSupported).flatMap { + case TryMergeResult(mergedChild, npMapping, cpMapping, None, None) => + val mappedNPGroupingExpression = + np.groupingExpressions.map(mapAttributes(_, npMapping)) + val mappedCPGroupingExpression = + cp.groupingExpressions.map(mapAttributes(_, cpMapping)) + // 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 (mappedNPGroupingExpression.map(_.canonicalized) == + mappedCPGroupingExpression.map(_.canonicalized)) { + val (mergedAggregateExpressions, newNPMapping, newCPMapping) = + mergeNamedExpressions(np.aggregateExpressions, cp.aggregateExpressions, npMapping, + cpMapping) + val mergedPlan = + Aggregate(mappedCPGroupingExpression, mergedAggregateExpressions, mergedChild) + Some(TryMergeResult(mergedPlan, newNPMapping, newCPMapping)) + } else { + None + } + case TryMergeResult(mergedChild, npMapping, cpMapping, npFilterOpt, cpFilterOpt) => + // childFilterPropagationSupported guarantees both aggregates have no grouping, so + // the grouping-match check is skipped. + assert(childFilterPropagationSupported) - 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 - } + // Apply each propagated boolean attribute as a FILTER (WHERE ...) clause on the + // corresponding side's aggregate expressions. + // A None filter means the side's aggregate expressions already carry their individual + // FILTER attributes from a previous merge round and should be left unchanged. + // Filter propagation is consumed here and not passed further up. + val filteredNPAggregateExpressions = npFilterOpt.fold(np.aggregateExpressions) { + case (f, _) => applyFilterToAggregateExpressions(np.aggregateExpressions, f) + } + val filteredCPAggregateExpressions = cpFilterOpt.fold(cp.aggregateExpressions)( + applyFilterToAggregateExpressions(cp.aggregateExpressions, _)) + val (mergedAggregateExpressions, newNPMapping, newCPMapping) = + mergeNamedExpressions(filteredNPAggregateExpressions, + filteredCPAggregateExpressions, npMapping, cpMapping) + val mergedPlan = Aggregate(Seq.empty, mergedAggregateExpressions, mergedChild) + Some(TryMergeResult(mergedPlan, newNPMapping, newCPMapping)) } - 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)) + case (np: Filter, cp: Filter) => + tryMergePlans(np.child, cp.child, filterPropagationSupported).flatMap { + case TryMergeResult(mergedChild, npMapping, cpMapping, npFilter, cpFilter) => + val mappedNPCondition = mapAttributes(np.condition, npMapping) + val mappedCPCondition = mapAttributes(cp.condition, cpMapping) // Comparing the canonicalized form is required to ignore different forms of the same - // expression and `AttributeReference.qualifier`s in `cp.condition`. - if (mappedNewCondition.map(_.canonicalized) == cp.condition.map(_.canonicalized)) { - val mergedPlan = cp.withNewChildren(Seq(mergedLeft, mergedRight)) - Some(mergedPlan -> outputMap) + // expression. + if (mappedNPCondition.canonicalized == mappedCPCondition.canonicalized) { + // Identical conditions: the filter node itself adds no new discrimination between + // the two sides, so we keep it unchanged and pass the child's mappings up. + val mergedPlan = Filter(mappedCPCondition, mergedChild) + Some(TryMergeResult(mergedPlan, npMapping, cpMapping, npFilter, cpFilter)) + } else if (filterPropagationSupported && symmetricFilterPropagationEnabled) { + if (cp.getTagValue(PlanMerger.MERGED_FILTER_TAG).isDefined) { + // cp Filter is already a merged filter from a previous round: its condition + // is OR(f0, f1, ...) and its child Project already contains aliases for those + // attributes. Only create a new alias for the np side, and extend the OR + // condition. + val newNPCondition = npFilter.fold(mappedNPCondition) { + case (f, _) => And(f, mappedNPCondition) + } + val childProject = mergedChild.asInstanceOf[Project] + // If newNPCondition is already aliased in the child Project (e.g. a third + // subplan whose filter matches one from a previous merge round), reuse the + // existing attribute instead of creating a redundant alias. + val existingNPFilter = childProject.projectList.collectFirst { + case a: Alias if a.child.canonicalized == newNPCondition.canonicalized => + a.toAttribute + } + existingNPFilter match { + case Some(reusedFilter) => + Some(TryMergeResult(cp, npMapping, cpMapping, Some((reusedFilter, false)), + None)) + case None => + val newNPFilterAlias = + Alias(newNPCondition, s"propagatedFilter_${PlanMerger.newId}")() + val newNPFilter = newNPFilterAlias.toAttribute + val newProject = childProject.copy( + projectList = childProject.projectList ++ Seq(newNPFilterAlias)) + val newFilter = Filter(Or(mappedCPCondition, newNPFilter), newProject) + newFilter.copyTagsFrom(cp) + Some(TryMergeResult(newFilter, npMapping, cpMapping, + Some((newNPFilter, true)), None)) + } + } else { + // First-time filter propagation: alias both sides' conditions as boolean + // attributes in a new Project below the Filter, and set the Filter condition + // to OR(newNPFilter, newCPFilter). + // Note: the new Project always uses mergedChild as its child (rather than + // flattening into an existing Project below) because mergedChild.output may + // contain previously-propagated filter attributes that newCPCondition + // references. + val newNPCondition = npFilter.fold(mappedNPCondition) { + case (f, _) => And(f, mappedNPCondition) + } + val newCPCondition = cpFilter.fold(mappedCPCondition)(And(_, mappedCPCondition)) + val newNPFilterAlias = + Alias(newNPCondition, s"propagatedFilter_${PlanMerger.newId}")() + val newCPFilterAlias = + Alias(newCPCondition, s"propagatedFilter_${PlanMerger.newId}")() + val newNPFilter = newNPFilterAlias.toAttribute + val newCPFilter = newCPFilterAlias.toAttribute + val project = Project( + mergedChild.output.toList ++ Seq(newNPFilterAlias, newCPFilterAlias), + mergedChild) + val newFilter = Filter(Or(newNPFilter, newCPFilter), project) + newFilter.copyTagsFrom(cp) + newFilter.setTagValue(PlanMerger.MERGED_FILTER_TAG, ()) + Some(TryMergeResult(newFilter, npMapping, cpMapping, Some((newNPFilter, true)), + Some(newCPFilter))) + } } else { None } - } + } + case (np: Filter, cp) if filterPropagationSupported => + tryMergePlans(np.child, cp, filterPropagationSupported).collect { + // If the cp side already propagated a filter from deeper recursion, the merge is + // effectively symmetric (both sides have a filter condition). Abort unless + // symmetricFilterPropagationEnabled. + case TryMergeResult(mergedChild, npMapping, cpMapping, npFilter, cpFilter) + if cpFilter.isEmpty || symmetricFilterPropagationEnabled => + val mappedNPCondition = mapAttributes(np.condition, npMapping) + val newNPCondition = npFilter.fold(mappedNPCondition) { + case (f, _) => And(f, mappedNPCondition) + } + val newNPFilterAlias = + Alias(newNPCondition, s"propagatedFilter_${PlanMerger.newId}")() + val newNPFilter = newNPFilterAlias.toAttribute + val project = Project( + mergedChild.output.toList ++ Seq(newNPFilterAlias) ++ cpFilter.toSeq, + mergedChild) + TryMergeResult(project, npMapping, cpMapping, Some((newNPFilter, true)), cpFilter) + } + case (np, cp: Filter) if filterPropagationSupported => + tryMergePlans(np, cp.child, filterPropagationSupported).collect { + // If the np side already propagated a filter from deeper recursion, the merge is + // effectively symmetric (both sides have a filter condition). Abort unless + // symmetricFilterPropagationEnabled. + case TryMergeResult(mergedChild, npMapping, cpMapping, npFilter, cpFilter) + if npFilter.isEmpty || symmetricFilterPropagationEnabled => + val mappedCPCondition = mapAttributes(cp.condition, cpMapping) + val newCPCondition = cpFilter.fold(mappedCPCondition)(And(_, mappedCPCondition)) + val newCPFilterAlias = + Alias(newCPCondition, s"propagatedFilter_${PlanMerger.newId}")() + val newCPFilter = newCPFilterAlias.toAttribute + val project = Project( + mergedChild.output.toList ++ npFilter.map(_._1).toSeq ++ Seq(newCPFilterAlias), + mergedChild) + TryMergeResult(project, npMapping, cpMapping, npFilter, Some(newCPFilter)) + } + + case (np: Join, cp: Join) if np.joinType == cp.joinType && np.hint == cp.hint => + // Filter propagation across joins is not yet supported. + tryMergePlans(np.left, cp.left, false).flatMap { + case TryMergeResult(mergedLeft, leftNPMapping, _, None, None) => + tryMergePlans(np.right, cp.right, false).flatMap { + case TryMergeResult(mergedRight, rightNPMapping, _, None, None) => + val npMapping = leftNPMapping ++ rightNPMapping + val mappedNPCondition = np.condition.map(mapAttributes(_, npMapping)) + // Comparing the canonicalized form is required to ignore different forms of the + // same expression and `AttributeReference.qualifier`s in `cp.condition`. + if (mappedNPCondition.map(_.canonicalized) == cp.condition.map(_.canonicalized)) { + val mergedPlan = cp.withNewChildren(Seq(mergedLeft, mergedRight)) + Some(TryMergeResult(mergedPlan, npMapping)) + } else { + None + } + case _ => None + } + case _ => None } // Otherwise merging is not possible. @@ -247,29 +460,107 @@ class PlanMerger { }.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. + // Remaps attributes of `newPlanExpressions` through `newPlanMapping` and attributes of + // `cachedPlanExpressions` through `cachedPlanMapping`, then merges them into a single + // expression list. + // Returns a triple of: + // 1. The merged expression list + // 2. New plan output map: ne.toAttribute -> merged plan attr (for parent nodes to remap + // new-plan-side expressions) + // 3. Cached plan output map: old wrapped cached attr -> new alias attr (for parent nodes to + // remap cached-plan-side expressions that would otherwise reference stale attributes after + // wrapping). Empty when no cached expressions were wrapped. + // + // When `newPlanFilter`/`cachedPlanFilter` are provided (filter propagation active), non-matching + // expressions from each side are wrapped with `If(filterAttr, expr, null)`. This ensures that a + // non-matching expression from one side evaluates to null for rows that belong to the other side, + // which is safe for aggregate FILTER (WHERE ...) semantics and avoids computing values for + // irrelevant rows. The filter attributes themselves are appended to the merged expression list so + // they remain visible to the enclosing Aggregate that will consume them. A newPlanFilter with + // isNew=false was reused from a previous merge round and is already present in the merged child + // output, so it is not appended again. 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) + newPlanExpressions: Seq[NamedExpression], + cachedPlanExpressions: Seq[NamedExpression], + newPlanMapping: AttributeMap[Attribute], + cachedPlanMapping: AttributeMap[Attribute] = AttributeMap.empty, + newPlanFilter: Option[(Attribute, Boolean)] = None, + cachedPlanFilter: Option[Attribute] = None) = { + val mergedExpressions = mutable.ArrayBuffer[NamedExpression]( + cachedPlanExpressions.map(mapAttributes(_, cachedPlanMapping)): _*) + val matchedCachedIndices = mutable.HashSet.empty[Int] + val newNPMapping = AttributeMap(newPlanExpressions.map { ne => + val mapped = mapAttributes(ne, newPlanMapping) val withoutAlias = mapped match { case Alias(child, _) => child case e => e } - ne.toAttribute -> mergedExpressions.find { + val foundIdx = mergedExpressions.indexWhere { case Alias(child, _) => child semanticEquals withoutAlias case e => e semanticEquals withoutAlias - }.getOrElse { - mergedExpressions += mapped - mapped - }.toAttribute + } + val resultAttr = if (foundIdx >= 0) { + // Matching expression: both sides compute the same value, no wrapping needed. + matchedCachedIndices += foundIdx + mergedExpressions(foundIdx).toAttribute + } else { + // Non-matching expression from the new plan side: wrap with the new plan filter so it + // is only computed for rows that belong to the new plan side. Plain attribute references + // are not wrapped since reading a column value is free. + val wrappedExpr: NamedExpression = newPlanFilter match { + case Some((f, _)) if !withoutAlias.isInstanceOf[Attribute] => + Alias(If(f, withoutAlias, Literal(null, withoutAlias.dataType)), mapped.name)() + case _ => mapped + } + mergedExpressions += wrappedExpr + wrappedExpr.toAttribute + } + ne.toAttribute -> resultAttr }) - (mergedExpressions.toSeq, newOutputMap) + + // Wrap unmatched cached expressions with the cached plan's filter so they are only computed + // for rows that belong to the cached plan side. Plain attribute references are not wrapped. + // Record each attr rewrite in the cached plan map so ancestor nodes can remap their stale + // references. + val newCPMapping = AttributeMap(cachedPlanFilter.toSeq.flatMap { f => + mergedExpressions.zipWithIndex.flatMap { + case (ce, i) if !matchedCachedIndices.contains(i) => + val withoutAlias = ce match { + case Alias(child, _) => child + case e => e + } + // Plain attribute references are not wrapped: no remapping entry needed. + Option.when(!withoutAlias.isInstanceOf[Attribute]) { + val newAlias = + Alias(If(f, withoutAlias, Literal(null, withoutAlias.dataType)), ce.name)() + mergedExpressions(i) = newAlias + ce.toAttribute -> newAlias.toAttribute + } + case _ => None + } + }) + + newPlanFilter.foreach { + case (f, true) => mergedExpressions += f + case _ => + } + cachedPlanFilter.foreach(mergedExpressions += _) + + (mergedExpressions.toSeq, newNPMapping, newCPMapping) + } + + // Applies filter as a FILTER (WHERE ...) clause to every AggregateExpression in exprs, + // combining with any pre-existing filter on the aggregate via AND. + private def applyFilterToAggregateExpressions( + exprs: Seq[NamedExpression], + filter: Attribute): Seq[NamedExpression] = { + exprs.map(_.transform { + case ae: AggregateExpression => + val combinedFilter = ae.filter.fold[Expression](filter)(And(filter, _)) + val newAE = ae.copy(filter = Some(combinedFilter)) + newAE.copyTagsFrom(ae) + newAE + }.asInstanceOf[NamedExpression]) } // Only allow aggregates of the same implementation because merging different implementations diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3297e4ef99e46..58325882b90c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -6487,6 +6487,31 @@ object SQLConf { .booleanConf .createOptional + val PLAN_MERGE_FILTER_PROPAGATION_ENABLED = + buildConf("spark.sql.planMerge.filterPropagation.enabled") + .internal() + .doc("When set to true, subquery plans that differ only in their filter conditions can " + + "be merged by propagating filters up to enclosing non-grouping aggregates.") + .version("4.2.0") + .withBindingPolicy(ConfigBindingPolicy.SESSION) + .booleanConf + .createWithDefault(true) + + val PLAN_MERGE_SYMMETRIC_FILTER_PROPAGATION_ENABLED = + buildConf("spark.sql.planMerge.symmetricFilterPropagation.enabled") + .doc("When set to true, two non-grouping aggregate subplans that both have filter " + + "conditions (but with different predicates) can be merged into a single scan using " + + "FILTER (WHERE ...) clauses on each aggregate expression. " + + "Merging two filtered scans broadens the combined filter to OR(f1, f2), which may " + + "reduce IO pruning (e.g. partition or file skipping) compared to the individual " + + "filters. In most cases the saving from a single scan outweighs this, but on highly " + + "partitioned or file-pruned tables disabling this option can improve performance. " + + s"Has no effect when ${PLAN_MERGE_FILTER_PROPAGATION_ENABLED.key} is false.") + .version("4.2.0") + .withBindingPolicy(ConfigBindingPolicy.SESSION) + .booleanConf + .createWithDefault(true) + val ERROR_MESSAGE_FORMAT = buildConf("spark.sql.error.messageFormat") .doc("When PRETTY, the error message consists of textual representation of error class, " + "message and query context. Stack traces are only shown for internal errors " + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeSubplansSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeSubplansSuite.scala index b368035e278eb..95ad5b0fa3202 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeSubplansSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeSubplansSuite.scala @@ -19,16 +19,18 @@ 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.expressions.{Alias, And, Attribute, CreateNamedStruct, GetStructField, Literal, Or, ScalarSubquery} +import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, CollectSet, Max, Min, Sum} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.internal.SQLConf class MergeSubplansSuite extends PlanTest { override def beforeEach(): Unit = { CTERelationDef.curId.set(0) + PlanMerger.curId.set(0) } private object Optimize extends RuleExecutor[LogicalPlan] { @@ -721,4 +723,652 @@ class MergeSubplansSuite extends PlanTest { comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } + + test("SPARK-40193: Merge non-grouping subqueries with different filter conditions") { + val subquery1 = ScalarSubquery(testRelation.where($"a" > 1).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"a" < 1).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + val npFilterAlias = Alias($"a" < 1, "propagatedFilter_0")() + val cpFilterAlias = Alias($"a" > 1, "propagatedFilter_1")() + val npFilter = npFilterAlias.toAttribute + val cpFilter = cpFilterAlias.toAttribute + val mergedSubquery = testRelation + .select((testRelation.output ++ Seq(npFilterAlias, cpFilterAlias)): _*) + .where(Or(npFilter, cpFilter)) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(cpFilter)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(npFilter)).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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("SPARK-40193: Merge three non-grouping subqueries with different filter conditions") { + val subquery1 = ScalarSubquery(testRelation.where($"a" > 1).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"a" < 1).groupBy()(min($"a").as("min_a"))) + val subquery3 = ScalarSubquery(testRelation.where($"a" === 1).groupBy()(sum($"a").as("sum_a"))) + val originalQuery = testRelation.select(subquery1, subquery2, subquery3) + + // Step 1: subquery1 (cp) and subquery2 (np) merge: + // f0 = Alias(a < 1, "propagatedFilter_0") -- np / min + // f1 = Alias(a > 1, "propagatedFilter_1") -- cp / max + // -> Project([a,b,c, f0Alias, f1Alias], testRelation) + // -> Filter(OR(f0, f1), above) [tagged] + // propagates (Some(f0), Some(f1)) upward + // + // Step 2: subquery3 (np) merges with merged(1,2) (cp). The cp Filter is tagged, so only a + // new np alias is created and flattened into the existing Project (no nested Projects): + // f2 = Alias(a === 1, "propagatedFilter_2") -- np / sum + // -> Project([a,b,c, f0Alias, f1Alias, f2Alias], testRelation) + // -> Filter(OR(OR(f0, f1), f2), above) [tagged] + // propagates (Some(f2), None) upward + // + // Aggregate: cp agg expressions already carry their FILTERs from step 1 and are unchanged. + // max(a) FILTER f1 -- a > 1 + // min(a) FILTER f0 -- a < 1 + // sum(a) FILTER f2 -- a === 1 + val npFilter0Alias = Alias($"a" < 1, "propagatedFilter_0")() + val cpFilter0Alias = Alias($"a" > 1, "propagatedFilter_1")() + val npFilter0 = npFilter0Alias.toAttribute + val cpFilter0 = cpFilter0Alias.toAttribute + val npFilter1Alias = Alias($"a" === 1, "propagatedFilter_2")() + val npFilter1 = npFilter1Alias.toAttribute + val mergedSubquery = testRelation + .select((testRelation.output ++ Seq(npFilter0Alias, cpFilter0Alias, npFilter1Alias)): _*) + .where(Or(Or(npFilter0, cpFilter0), npFilter1)) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(cpFilter0)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(npFilter0)).as("min_a"), + Sum($"a").toAggregateExpression(isDistinct = false, filter = Some(npFilter1)).as("sum_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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), + extractorExpression(0, analyzedMergedSubquery.output, 2)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("SPARK-40193: Merge three non-grouping subqueries where the third has the same filter " + + "condition as the first") { + val subquery1 = ScalarSubquery(testRelation.where($"a" > 1).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"a" < 1).groupBy()(min($"a").as("min_a"))) + val subquery3 = ScalarSubquery(testRelation.where($"a" > 1).groupBy()(sum($"a").as("sum_a"))) + val originalQuery = testRelation.select(subquery1, subquery2, subquery3) + + // Step 1: subquery1 (cp) and subquery2 (np) merge as usual: + // f0 = Alias(a < 1, "propagatedFilter_0") -- np / min + // f1 = Alias(a > 1, "propagatedFilter_1") -- cp / max + // -> Project([a,b,c, f0Alias, f1Alias], testRelation) + // -> Filter(OR(f0, f1), above) [tagged] + // + // Step 2: subquery3 (np, condition a > 1) merges with merged(1,2) (cp). The cp Filter is + // tagged and (a > 1) is already aliased as f1 in the child Project, so f1 is reused and no + // new alias or extended OR condition is created. Only sum(a) FILTER f1 is added to the agg. + val f0Alias = Alias($"a" < 1, "propagatedFilter_0")() + val f1Alias = Alias($"a" > 1, "propagatedFilter_1")() + val f0 = f0Alias.toAttribute + val f1 = f1Alias.toAttribute + val mergedSubquery = testRelation + .select((testRelation.output ++ Seq(f0Alias, f1Alias)): _*) + .where(Or(f0, f1)) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(f1)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(f0)).as("min_a"), + Sum($"a").toAggregateExpression(isDistinct = false, filter = Some(f1)).as("sum_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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), + extractorExpression(0, analyzedMergedSubquery.output, 2)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("SPARK-40193: Do not merge non-grouping subqueries with different filter conditions when " + + "disabled") { + withSQLConf(SQLConf.PLAN_MERGE_FILTER_PROPAGATION_ENABLED.key -> "false") { + val subquery1 = ScalarSubquery(testRelation.where($"a" > 1).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"a" < 1).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } + } + + test("SPARK-40193: Do not merge non-grouping subqueries with different filter conditions on " + + "both sides when symmetric filter propagation is disabled") { + withSQLConf(SQLConf.PLAN_MERGE_SYMMETRIC_FILTER_PROPAGATION_ENABLED.key -> "false") { + val subquery1 = ScalarSubquery(testRelation.where($"a" > 1).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"a" < 1).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } + } + + test("SPARK-40193: Merge non-grouping aggregates with different filter conditions") { + val agg1 = testRelation.where($"a" > 1).groupBy()(max($"a").as("max_a")) + val agg2 = testRelation.where($"a" < 1).groupBy()(min($"a").as("min_a")) + val originalQuery = agg1.join(agg2) + + val npFilterAlias = Alias($"a" < 1, "propagatedFilter_0")() + val cpFilterAlias = Alias($"a" > 1, "propagatedFilter_1")() + val npFilter = npFilterAlias.toAttribute + val cpFilter = cpFilterAlias.toAttribute + val mergedSubquery = testRelation + .select((testRelation.output ++ Seq(npFilterAlias, cpFilterAlias)): _*) + .where(Or(npFilter, cpFilter)) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(cpFilter)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(npFilter)).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_a" + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + OneRowRelation() + .select(extractorExpression(0, analyzedMergedSubquery.output, 0, "max_a")) + .join( + OneRowRelation() + .select(extractorExpression(0, analyzedMergedSubquery.output, 1, "min_a"))), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("SPARK-40193: Do not merge grouping aggregates with different filter conditions") { + val subquery1 = ScalarSubquery( + testRelation.where($"a" > 1).groupBy($"b")(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery( + testRelation.where($"a" < 1).groupBy($"b")(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } + + test("SPARK-40193: Merge non-grouping subqueries where only the new plan has a filter") { + val subquery1 = ScalarSubquery(testRelation.groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"a" < 1).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + val npFilterAlias = Alias($"a" < 1, "propagatedFilter_0")() + val npFilter = npFilterAlias.toAttribute + val mergedSubquery = testRelation + .select((testRelation.output ++ Seq(npFilterAlias)): _*) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(npFilter)).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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("SPARK-40193: Merge non-grouping subqueries where only the cached plan has a filter") { + val subquery1 = ScalarSubquery(testRelation.where($"a" > 1).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + val cpFilterAlias = Alias($"a" > 1, "propagatedFilter_0")() + val cpFilter = cpFilterAlias.toAttribute + val mergedSubquery = testRelation + .select((testRelation.output ++ Seq(cpFilterAlias)): _*) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(cpFilter)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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("SPARK-40193: Merge non-grouping subqueries with multiple stacked filter conditions") { + val subquery1 = ScalarSubquery( + testRelation.where($"a" > 1).where($"b" > 2).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery( + testRelation.where($"a" < 1).where($"b" < 2).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + // Merge traversal (inner-to-outer): + // + // Inner level - (np: Filter(a < 1), cp: Filter(a > 1)): + // f0 = Alias(a < 1, "propagatedFilter_0") -- np / min + // f1 = Alias(a > 1, "propagatedFilter_1") -- cp / max + // -> Project([a,b,c, f0_alias, f1_alias], testRelation) + // -> Filter(OR(f0, f1), above) [tagged] + // propagates (Some(f0), Some(f1)) upward + // + // Outer level - (np: Filter(b < 2), cp: Filter(b > 2)): + // f2 = Alias(AND(f0, b < 2), "propagatedFilter_2") -- np + // f3 = Alias(AND(f1, b > 2), "propagatedFilter_3") -- cp + // -> Project([a,b,c, f0, f1, f2_alias, f3_alias], innerFilter) + // -> Filter(OR(f2, f3), above) [tagged] + // propagates (Some(f2), Some(f3)) upward + // + // Aggregate consumes f2/f3 as FILTER clauses: + // max(a) FILTER f3 -- AND(a > 1, b > 2) + // min(a) FILTER f2 -- AND(a < 1, b < 2) + val f0Alias = Alias($"a" < 1, "propagatedFilter_0")() + val f1Alias = Alias($"a" > 1, "propagatedFilter_1")() + val f0 = f0Alias.toAttribute + val f1 = f1Alias.toAttribute + val innerProject = testRelation.select((testRelation.output ++ Seq(f0Alias, f1Alias)): _*) + val innerFilter = innerProject.where(Or(f0, f1)) + val f2Alias = Alias(And(f0, $"b" < 2), "propagatedFilter_2")() + val f3Alias = Alias(And(f1, $"b" > 2), "propagatedFilter_3")() + val f2 = f2Alias.toAttribute + val f3 = f3Alias.toAttribute + val mergedSubquery = innerFilter + .select((innerFilter.output ++ Seq(f2Alias, f3Alias)): _*) + .where(Or(f2, f3)) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(f3)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(f2)).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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("SPARK-40193: Merge non-grouping subqueries where the new plan has more filter layers") { + val subquery1 = ScalarSubquery(testRelation.where($"a" > 1).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery( + testRelation.where($"a" < 1).where($"b" < 2).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + // Merge traversal (inner-to-outer): + // + // Inner level - (np: Filter(a < 1), cp: testRelation): + // cp has no filter -> (np: Filter, cp) case. No Filter node needed. + // f0 = Alias(a < 1, "propagatedFilter_0") + // -> Project([a, b, c, f0Alias], testRelation) + // propagates (Some(f0), None) upward + // + // Outer level - (np: Filter(b < 2), cp: Filter(a > 1)): + // Both are Filters. Child result has (npFilter=Some(f0), cpFilter=None). + // f1 = Alias(AND(f0, b < 2), "propagatedFilter_1") -- np combined condition + // f2 = Alias(a > 1, "propagatedFilter_2") -- cp condition + // -> Project([a, b, c, f0, f1Alias, f2Alias], innerProject) + // -> Filter(OR(f1, f2), above) [tagged] + // propagates (Some(f1), Some(f2)) upward + // + // Aggregate: + // max(a) FILTER f2 -- cp: a > 1 + // min(a) FILTER f1 -- np: a < 1 AND b < 2 + val f0Alias = Alias($"a" < 1, "propagatedFilter_0")() + val f0 = f0Alias.toAttribute + val innerProject = testRelation.select((testRelation.output ++ Seq(f0Alias)): _*) + val f1Alias = Alias(And(f0, $"b" < 2), "propagatedFilter_1")() + val f2Alias = Alias($"a" > 1, "propagatedFilter_2")() + val f1 = f1Alias.toAttribute + val f2 = f2Alias.toAttribute + val mergedSubquery = innerProject + .select((innerProject.output ++ Seq(f1Alias, f2Alias)): _*) + .where(Or(f1, f2)) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(f2)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(f1)).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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("SPARK-40193: Merge non-grouping subqueries where the cached plan has more filter layers") { + val subquery1 = ScalarSubquery( + testRelation.where($"a" > 1).where($"b" > 2).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"a" < 1).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + // Merge traversal (inner-to-outer): + // + // Inner level - (np: testRelation, cp: Filter(a > 1)): + // np has no filter -> (np, cp: Filter) case. No Filter node needed. + // f0 = Alias(a > 1, "propagatedFilter_0") + // -> Project([a, b, c, f0Alias], testRelation) + // propagates (None, Some(f0)) upward + // + // Outer level - (np: Filter(a < 1), cp: Filter(b > 2)): + // Both are Filters. Child result has (npFilter=None, cpFilter=Some(f0)). + // f1 = Alias(a < 1, "propagatedFilter_1") -- np condition + // f2 = Alias(AND(f0, b > 2), "propagatedFilter_2") -- cp combined condition + // -> Project([a, b, c, f0, f1Alias, f2Alias], innerProject) + // -> Filter(OR(f1, f2), above) [tagged] + // propagates (Some(f1), Some(f2)) upward + // + // Aggregate: + // max(a) FILTER f2 -- cp: a > 1 AND b > 2 + // min(a) FILTER f1 -- np: a < 1 + val f0Alias = Alias($"a" > 1, "propagatedFilter_0")() + val f0 = f0Alias.toAttribute + val innerProject = testRelation.select((testRelation.output ++ Seq(f0Alias)): _*) + val f1Alias = Alias($"a" < 1, "propagatedFilter_1")() + val f2Alias = Alias(And(f0, $"b" > 2), "propagatedFilter_2")() + val f1 = f1Alias.toAttribute + val f2 = f2Alias.toAttribute + val mergedSubquery = innerProject + .select((innerProject.output ++ Seq(f1Alias, f2Alias)): _*) + .where(Or(f1, f2)) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(f2)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(f1)).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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("SPARK-40193: Merge non-grouping subqueries with equal outer stacked filter") { + val subquery1 = ScalarSubquery( + testRelation.where($"a" > 1).where($"b" > 2).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery( + testRelation.where($"a" < 1).where($"b" > 2).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + // Merge traversal (inner-to-outer): + // + // Inner level - (np: Filter(a < 1), cp: Filter(a > 1)): + // Different conditions -> first-time filter propagation. + // f0 = Alias(a < 1, "propagatedFilter_0") -- np + // f1 = Alias(a > 1, "propagatedFilter_1") -- cp + // -> Project([a, b, c, f0Alias, f1Alias], testRelation) + // -> Filter(OR(f0, f1)) [tagged] + // propagates (Some(f0), Some(f1)) upward + // + // Outer level - (np: Filter(b > 2), cp: Filter(b > 2)): + // Equal conditions -> Filter(b > 2, ...) passes filter attrs through. + // propagates (Some(f0), Some(f1)) unchanged + // + // Aggregate: + // max(a) FILTER f1 -- cp: a > 1 (plus the outer b > 2 applied to all rows) + // min(a) FILTER f0 -- np: a < 1 (plus the outer b > 2 applied to all rows) + val f0Alias = Alias($"a" < 1, "propagatedFilter_0")() + val f1Alias = Alias($"a" > 1, "propagatedFilter_1")() + val f0 = f0Alias.toAttribute + val f1 = f1Alias.toAttribute + val innerProject = testRelation.select((testRelation.output ++ Seq(f0Alias, f1Alias)): _*) + val innerFilter = innerProject.where(Or(f0, f1)) + val mergedSubquery = innerFilter + .where($"b" > 2) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(f1)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(f0)).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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("SPARK-40193: Merge non-grouping subqueries with equal inner stacked filter") { + val subquery1 = ScalarSubquery( + testRelation.where($"a" > 1).where($"b" > 2).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery( + testRelation.where($"a" > 1).where($"b" < 2).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + // Merge traversal (inner-to-outer): + // + // Inner level - (np: Filter(a > 1), cp: Filter(a > 1)): + // checkIdenticalPlans matches -> no filter propagation needed. + // -> Filter(a > 1, testRelation) (shared, unchanged) + // propagates (None, None) upward + // + // Outer level - (np: Filter(b < 2), cp: Filter(b > 2)): + // Different conditions -> first-time filter propagation. + // f0 = Alias(b < 2, "propagatedFilter_0") -- np + // f1 = Alias(b > 2, "propagatedFilter_1") -- cp + // -> Project([a, b, c, f0Alias, f1Alias], Filter(a > 1, testRelation)) + // -> Filter(OR(f0, f1)) [tagged] + // propagates (Some(f0), Some(f1)) upward + // + // Aggregate: + // max(a) FILTER f1 -- cp: a > 1 AND b > 2 + // min(a) FILTER f0 -- np: a > 1 AND b < 2 + val f0Alias = Alias($"b" < 2, "propagatedFilter_0")() + val f1Alias = Alias($"b" > 2, "propagatedFilter_1")() + val f0 = f0Alias.toAttribute + val f1 = f1Alias.toAttribute + val innerFilter = testRelation.where($"a" > 1) + val mergedSubquery = innerFilter + .select((innerFilter.output ++ Seq(f0Alias, f1Alias)): _*) + .where(Or(f0, f1)) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(f1)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(f0)).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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("SPARK-40193: Merge non-grouping subqueries where the new plan has an extra inner filter " + + "below a shared outer filter") { + val subquery1 = ScalarSubquery(testRelation.where($"a" > 1).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery( + testRelation.where($"b" < 2).where($"a" > 1).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + // Merge traversal (inner-to-outer): + // + // Inner level - (np: Filter(b < 2), cp: testRelation): + // cp has no filter -> (np: Filter, cp) case. No Filter node needed. + // f0 = Alias(b < 2, "propagatedFilter_0") + // -> Project([a, b, c, f0Alias], testRelation) + // propagates (Some(f0), None) upward + // + // Outer level - (np: Filter(a > 1), cp: Filter(a > 1)): + // Equal conditions -> just wraps with Filter(a > 1, ...) and passes filter attrs through. + // propagates (Some(f0), None) unchanged + // + // Aggregate: + // max(a) unfiltered -- cp: all rows where a > 1 (from outer Filter) + // min(a) FILTER f0 -- np: rows where a > 1 AND b < 2 + val f0Alias = Alias($"b" < 2, "propagatedFilter_0")() + val f0 = f0Alias.toAttribute + val innerProject = testRelation.select((testRelation.output ++ Seq(f0Alias)): _*) + val mergedSubquery = innerProject + .where($"a" > 1) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(f0)).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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("SPARK-40193: Merge non-grouping subqueries where the cached plan has an extra inner " + + "filter below a shared outer filter") { + val subquery1 = ScalarSubquery( + testRelation.where($"b" < 2).where($"a" > 1).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"a" > 1).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + // Merge traversal (inner-to-outer): + // + // Inner level - (np: testRelation, cp: Filter(b < 2)): + // np has no filter -> (np, cp: Filter) case. No Filter node needed. + // f0 = Alias(b < 2, "propagatedFilter_0") + // -> Project([a, b, c, f0Alias], testRelation) + // propagates (None, Some(f0)) upward + // + // Outer level - (np: Filter(a > 1), cp: Filter(a > 1)): + // Equal conditions -> just wraps with Filter(a > 1, ...) and passes filter attrs through. + // propagates (None, Some(f0)) unchanged + // + // Aggregate: + // max(a) FILTER f0 -- cp: rows where a > 1 AND b < 2 + // min(a) unfiltered -- np: all rows where a > 1 (from outer Filter) + val f0Alias = Alias($"b" < 2, "propagatedFilter_0")() + val f0 = f0Alias.toAttribute + val innerProject = testRelation.select((testRelation.output ++ Seq(f0Alias)): _*) + val mergedSubquery = innerProject + .where($"a" > 1) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(f0)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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("SPARK-40193: Merge non-grouping subqueries with equal conditions in reversed filter " + + "order") { + val subquery1 = ScalarSubquery( + testRelation.where($"a" > 1).where($"b" > 2).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery( + testRelation.where($"b" > 2).where($"a" > 1).groupBy()(min($"a").as("min_a"))) + val originalQuery = testRelation.select(subquery1, subquery2) + + // Merge traversal (inner-to-outer): + // + // Because the conditions are in opposite order, each pair of Filter nodes has different + // conditions and filter propagation is triggered at both levels, producing 4 filter + // attributes in total even though both sides ultimately encode a > 1 AND b > 2. + // + // Inner level - (np: Filter(b > 2), cp: Filter(a > 1)): + // f0 = Alias(b > 2, "propagatedFilter_0") -- np inner condition + // f1 = Alias(a > 1, "propagatedFilter_1") -- cp inner condition + // -> Project([a, b, c, f0Alias, f1Alias], testRelation) + // -> Filter(OR(f0, f1)) [tagged] + // propagates (Some(f0), Some(f1)) upward + // + // Outer level - (np: Filter(a > 1), cp: Filter(b > 2)): + // f2 = Alias(AND(f0, a > 1), "propagatedFilter_2") -- np: b > 2 AND a > 1 + // f3 = Alias(AND(f1, b > 2), "propagatedFilter_3") -- cp: a > 1 AND b > 2 + // -> Project([a, b, c, f0, f1, f2Alias, f3Alias], innerFilter) + // -> Filter(OR(f2, f3)) [tagged] + // propagates (Some(f2), Some(f3)) upward + // + // Aggregate: + // max(a) FILTER f3 -- cp: a > 1 AND b > 2 + // min(a) FILTER f2 -- np: b > 2 AND a > 1 (same predicate, different representation) + val f0Alias = Alias($"b" > 2, "propagatedFilter_0")() + val f1Alias = Alias($"a" > 1, "propagatedFilter_1")() + val f0 = f0Alias.toAttribute + val f1 = f1Alias.toAttribute + val innerProject = testRelation.select((testRelation.output ++ Seq(f0Alias, f1Alias)): _*) + val innerFilter = innerProject.where(Or(f0, f1)) + val f2Alias = Alias(And(f0, $"a" > 1), "propagatedFilter_2")() + val f3Alias = Alias(And(f1, $"b" > 2), "propagatedFilter_3")() + val f2 = f2Alias.toAttribute + val f3 = f3Alias.toAttribute + val mergedSubquery = innerFilter + .select((innerFilter.output ++ Seq(f2Alias, f3Alias)): _*) + .where(Or(f2, f3)) + .groupBy()( + Max($"a").toAggregateExpression(isDistinct = false, filter = Some(f3)).as("max_a"), + Min($"a").toAggregateExpression(isDistinct = false, filter = Some(f2)).as("min_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("min_a"), $"min_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) + } } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt index 0c4e41f43831d..17d9dd6d12c77 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt @@ -82,7 +82,7 @@ Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] (6) Filter [codegen id : 1] Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : ((isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) AND might_contain(Subquery scalar-subquery#7, [id=#1], xxhash64(d_week_seq#5, 42))) +Condition : ((isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) AND might_contain(Subquery scalar-subquery#7, [id=#1].bloomFilter, xxhash64(d_week_seq#5, 42))) (7) BroadcastExchange Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] @@ -203,11 +203,11 @@ Input [3]: [d_date_sk#54, d_week_seq#55, d_day_name#56] (31) Filter [codegen id : 5] Input [3]: [d_date_sk#54, d_week_seq#55, d_day_name#56] -Condition : ((isnotnull(d_date_sk#54) AND isnotnull(d_week_seq#55)) AND might_contain(Subquery scalar-subquery#57, [id=#6], xxhash64(d_week_seq#55, 42))) +Condition : ((isnotnull(d_date_sk#54) AND isnotnull(d_week_seq#55)) AND might_contain(ReusedSubquery Subquery scalar-subquery#7, [id=#1].bloomFilter, xxhash64(d_week_seq#55, 42))) (32) BroadcastExchange Input [3]: [d_date_sk#54, d_week_seq#55, d_day_name#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#53] @@ -223,115 +223,116 @@ Input [6]: [ss_store_sk#51, ss_sales_price#52, ss_sold_date_sk#53, d_date_sk#54, Input [4]: [ss_store_sk#51, ss_sales_price#52, d_week_seq#55, d_day_name#56] Keys [2]: [d_week_seq#55, ss_store_sk#51] Functions [6]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Sunday ) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Monday ) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Wednesday) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Thursday ) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Friday ) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Saturday ) THEN ss_sales_price#52 END))] -Aggregate Attributes [6]: [sum#58, sum#59, sum#60, sum#61, sum#62, sum#63] -Results [8]: [d_week_seq#55, ss_store_sk#51, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69] +Aggregate Attributes [6]: [sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] +Results [8]: [d_week_seq#55, ss_store_sk#51, sum#63, sum#64, sum#65, sum#66, sum#67, sum#68] (36) Exchange -Input [8]: [d_week_seq#55, ss_store_sk#51, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69] -Arguments: hashpartitioning(d_week_seq#55, ss_store_sk#51, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [8]: [d_week_seq#55, ss_store_sk#51, sum#63, sum#64, sum#65, sum#66, sum#67, sum#68] +Arguments: hashpartitioning(d_week_seq#55, ss_store_sk#51, 5), ENSURE_REQUIREMENTS, [plan_id=7] (37) HashAggregate [codegen id : 9] -Input [8]: [d_week_seq#55, ss_store_sk#51, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69] +Input [8]: [d_week_seq#55, ss_store_sk#51, sum#63, sum#64, sum#65, sum#66, sum#67, sum#68] Keys [2]: [d_week_seq#55, ss_store_sk#51] Functions [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#56 = Sunday ) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Monday ) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Wednesday) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Thursday ) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Friday ) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Saturday ) THEN ss_sales_price#52 END))] Aggregate Attributes [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#56 = Sunday ) THEN ss_sales_price#52 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Monday ) THEN ss_sales_price#52 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Wednesday) THEN ss_sales_price#52 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Thursday ) THEN ss_sales_price#52 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Friday ) THEN ss_sales_price#52 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Saturday ) THEN ss_sales_price#52 END))#28] -Results [8]: [d_week_seq#55, ss_store_sk#51, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Sunday ) THEN ss_sales_price#52 END))#22,17,2) AS sun_sales#70, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Monday ) THEN ss_sales_price#52 END))#23,17,2) AS mon_sales#71, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Wednesday) THEN ss_sales_price#52 END))#25,17,2) AS wed_sales#72, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Thursday ) THEN ss_sales_price#52 END))#26,17,2) AS thu_sales#73, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Friday ) THEN ss_sales_price#52 END))#27,17,2) AS fri_sales#74, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Saturday ) THEN ss_sales_price#52 END))#28,17,2) AS sat_sales#75] +Results [8]: [d_week_seq#55, ss_store_sk#51, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Sunday ) THEN ss_sales_price#52 END))#22,17,2) AS sun_sales#69, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Monday ) THEN ss_sales_price#52 END))#23,17,2) AS mon_sales#70, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Wednesday) THEN ss_sales_price#52 END))#25,17,2) AS wed_sales#71, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Thursday ) THEN ss_sales_price#52 END))#26,17,2) AS thu_sales#72, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Friday ) THEN ss_sales_price#52 END))#27,17,2) AS fri_sales#73, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Saturday ) THEN ss_sales_price#52 END))#28,17,2) AS sat_sales#74] (38) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#76, s_store_id#77] +Output [2]: [s_store_sk#75, s_store_id#76] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct (39) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#76, s_store_id#77] +Input [2]: [s_store_sk#75, s_store_id#76] (40) Filter [codegen id : 7] -Input [2]: [s_store_sk#76, s_store_id#77] -Condition : (isnotnull(s_store_sk#76) AND isnotnull(s_store_id#77)) +Input [2]: [s_store_sk#75, s_store_id#76] +Condition : (isnotnull(s_store_sk#75) AND isnotnull(s_store_id#76)) (41) BroadcastExchange -Input [2]: [s_store_sk#76, s_store_id#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +Input [2]: [s_store_sk#75, s_store_id#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#76] +Right keys [1]: [s_store_sk#75] Join type: Inner Join condition: None (43) Project [codegen id : 9] -Output [8]: [d_week_seq#55, sun_sales#70, mon_sales#71, wed_sales#72, thu_sales#73, fri_sales#74, sat_sales#75, s_store_id#77] -Input [10]: [d_week_seq#55, ss_store_sk#51, sun_sales#70, mon_sales#71, wed_sales#72, thu_sales#73, fri_sales#74, sat_sales#75, s_store_sk#76, s_store_id#77] +Output [8]: [d_week_seq#55, sun_sales#69, mon_sales#70, wed_sales#71, thu_sales#72, fri_sales#73, sat_sales#74, s_store_id#76] +Input [10]: [d_week_seq#55, ss_store_sk#51, sun_sales#69, mon_sales#70, wed_sales#71, thu_sales#72, fri_sales#73, sat_sales#74, s_store_sk#75, s_store_id#76] (44) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#78, d_week_seq#79] +Output [2]: [d_month_seq#77, d_week_seq#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1197), LessThanOrEqual(d_month_seq,1208), IsNotNull(d_week_seq)] ReadSchema: struct (45) ColumnarToRow [codegen id : 8] -Input [2]: [d_month_seq#78, d_week_seq#79] +Input [2]: [d_month_seq#77, d_week_seq#78] (46) Filter [codegen id : 8] -Input [2]: [d_month_seq#78, d_week_seq#79] -Condition : (((isnotnull(d_month_seq#78) AND (d_month_seq#78 >= 1197)) AND (d_month_seq#78 <= 1208)) AND isnotnull(d_week_seq#79)) +Input [2]: [d_month_seq#77, d_week_seq#78] +Condition : (((isnotnull(d_month_seq#77) AND (d_month_seq#77 >= 1197)) AND (d_month_seq#77 <= 1208)) AND isnotnull(d_week_seq#78)) (47) Project [codegen id : 8] -Output [1]: [d_week_seq#79] -Input [2]: [d_month_seq#78, d_week_seq#79] +Output [1]: [d_week_seq#78] +Input [2]: [d_month_seq#77, d_week_seq#78] (48) BroadcastExchange -Input [1]: [d_week_seq#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Input [1]: [d_week_seq#78] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] (49) BroadcastHashJoin [codegen id : 9] Left keys [1]: [d_week_seq#55] -Right keys [1]: [d_week_seq#79] +Right keys [1]: [d_week_seq#78] Join type: Inner Join condition: None (50) Project [codegen id : 9] -Output [8]: [d_week_seq#55 AS d_week_seq2#80, s_store_id#77 AS s_store_id2#81, sun_sales#70 AS sun_sales2#82, mon_sales#71 AS mon_sales2#83, wed_sales#72 AS wed_sales2#84, thu_sales#73 AS thu_sales2#85, fri_sales#74 AS fri_sales2#86, sat_sales#75 AS sat_sales2#87] -Input [9]: [d_week_seq#55, sun_sales#70, mon_sales#71, wed_sales#72, thu_sales#73, fri_sales#74, sat_sales#75, s_store_id#77, d_week_seq#79] +Output [8]: [d_week_seq#55 AS d_week_seq2#79, s_store_id#76 AS s_store_id2#80, sun_sales#69 AS sun_sales2#81, mon_sales#70 AS mon_sales2#82, wed_sales#71 AS wed_sales2#83, thu_sales#72 AS thu_sales2#84, fri_sales#73 AS fri_sales2#85, sat_sales#74 AS sat_sales2#86] +Input [9]: [d_week_seq#55, sun_sales#69, mon_sales#70, wed_sales#71, thu_sales#72, fri_sales#73, sat_sales#74, s_store_id#76, d_week_seq#78] (51) BroadcastExchange -Input [8]: [d_week_seq2#80, s_store_id2#81, sun_sales2#82, mon_sales2#83, wed_sales2#84, thu_sales2#85, fri_sales2#86, sat_sales2#87] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=11] +Input [8]: [d_week_seq2#79, s_store_id2#80, sun_sales2#81, mon_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=10] (52) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#43, d_week_seq1#42] -Right keys [2]: [s_store_id2#81, (d_week_seq2#80 - 52)] +Right keys [2]: [s_store_id2#80, (d_week_seq2#79 - 52)] Join type: Inner Join condition: None (53) Project [codegen id : 10] -Output [10]: [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1#44 / sun_sales2#82) AS (sun_sales1 / sun_sales2)#88, (mon_sales1#45 / mon_sales2#83) AS (mon_sales1 / mon_sales2)#89, (tue_sales1#46 / tue_sales1#46) AS (tue_sales1 / tue_sales1)#90, (wed_sales1#47 / wed_sales2#84) AS (wed_sales1 / wed_sales2)#91, (thu_sales1#48 / thu_sales2#85) AS (thu_sales1 / thu_sales2)#92, (fri_sales1#49 / fri_sales2#86) AS (fri_sales1 / fri_sales2)#93, (sat_sales1#50 / sat_sales2#87) AS (sat_sales1 / sat_sales2)#94] -Input [18]: [s_store_name1#41, d_week_seq1#42, s_store_id1#43, sun_sales1#44, mon_sales1#45, tue_sales1#46, wed_sales1#47, thu_sales1#48, fri_sales1#49, sat_sales1#50, d_week_seq2#80, s_store_id2#81, sun_sales2#82, mon_sales2#83, wed_sales2#84, thu_sales2#85, fri_sales2#86, sat_sales2#87] +Output [10]: [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1#44 / sun_sales2#81) AS (sun_sales1 / sun_sales2)#87, (mon_sales1#45 / mon_sales2#82) AS (mon_sales1 / mon_sales2)#88, (tue_sales1#46 / tue_sales1#46) AS (tue_sales1 / tue_sales1)#89, (wed_sales1#47 / wed_sales2#83) AS (wed_sales1 / wed_sales2)#90, (thu_sales1#48 / thu_sales2#84) AS (thu_sales1 / thu_sales2)#91, (fri_sales1#49 / fri_sales2#85) AS (fri_sales1 / fri_sales2)#92, (sat_sales1#50 / sat_sales2#86) AS (sat_sales1 / sat_sales2)#93] +Input [18]: [s_store_name1#41, d_week_seq1#42, s_store_id1#43, sun_sales1#44, mon_sales1#45, tue_sales1#46, wed_sales1#47, thu_sales1#48, fri_sales1#49, sat_sales1#50, d_week_seq2#79, s_store_id2#80, sun_sales2#81, mon_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] (54) TakeOrderedAndProject -Input [10]: [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1 / sun_sales2)#88, (mon_sales1 / mon_sales2)#89, (tue_sales1 / tue_sales1)#90, (wed_sales1 / wed_sales2)#91, (thu_sales1 / thu_sales2)#92, (fri_sales1 / fri_sales2)#93, (sat_sales1 / sat_sales2)#94] -Arguments: 100, [s_store_name1#41 ASC NULLS FIRST, s_store_id1#43 ASC NULLS FIRST, d_week_seq1#42 ASC NULLS FIRST], [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1 / sun_sales2)#88, (mon_sales1 / mon_sales2)#89, (tue_sales1 / tue_sales1)#90, (wed_sales1 / wed_sales2)#91, (thu_sales1 / thu_sales2)#92, (fri_sales1 / fri_sales2)#93, (sat_sales1 / sat_sales2)#94] +Input [10]: [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1 / sun_sales2)#87, (mon_sales1 / mon_sales2)#88, (tue_sales1 / tue_sales1)#89, (wed_sales1 / wed_sales2)#90, (thu_sales1 / thu_sales2)#91, (fri_sales1 / fri_sales2)#92, (sat_sales1 / sat_sales2)#93] +Arguments: 100, [s_store_name1#41 ASC NULLS FIRST, s_store_id1#43 ASC NULLS FIRST, d_week_seq1#42 ASC NULLS FIRST], [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1 / sun_sales2)#87, (mon_sales1 / mon_sales2)#88, (tue_sales1 / tue_sales1)#89, (wed_sales1 / wed_sales2)#90, (thu_sales1 / thu_sales2)#91, (fri_sales1 / fri_sales2)#92, (sat_sales1 / sat_sales2)#93] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery#7, [id=#1] -ObjectHashAggregate (61) -+- Exchange (60) - +- ObjectHashAggregate (59) - +- * Project (58) - +- * Filter (57) - +- * ColumnarToRow (56) - +- Scan parquet spark_catalog.default.date_dim (55) +* Project (62) ++- ObjectHashAggregate (61) + +- Exchange (60) + +- ObjectHashAggregate (59) + +- * Project (58) + +- * Filter (57) + +- * ColumnarToRow (56) + +- Scan parquet spark_catalog.default.date_dim (55) (55) Scan parquet spark_catalog.default.date_dim Output [2]: [d_month_seq#39, d_week_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1185), LessThanOrEqual(d_month_seq,1196), IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_week_seq), Or(And(GreaterThanOrEqual(d_month_seq,1197),LessThanOrEqual(d_month_seq,1208)),And(GreaterThanOrEqual(d_month_seq,1185),LessThanOrEqual(d_month_seq,1196)))] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] @@ -339,74 +340,34 @@ Input [2]: [d_month_seq#39, d_week_seq#40] (57) Filter [codegen id : 1] Input [2]: [d_month_seq#39, d_week_seq#40] -Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1185)) AND (d_month_seq#39 <= 1196)) AND isnotnull(d_week_seq#40)) +Condition : ((isnotnull(d_month_seq#39) AND isnotnull(d_week_seq#40)) AND (((d_month_seq#39 >= 1197) AND (d_month_seq#39 <= 1208)) OR ((d_month_seq#39 >= 1185) AND (d_month_seq#39 <= 1196)))) (58) Project [codegen id : 1] -Output [1]: [d_week_seq#40] +Output [3]: [d_week_seq#40, ((isnotnull(d_month_seq#39) AND ((d_month_seq#39 >= 1197) AND (d_month_seq#39 <= 1208))) AND isnotnull(d_week_seq#40)) AS propagatedFilter_1#94, ((isnotnull(d_month_seq#39) AND ((d_month_seq#39 >= 1185) AND (d_month_seq#39 <= 1196))) AND isnotnull(d_week_seq#40)) AS propagatedFilter_2#95] Input [2]: [d_month_seq#39, d_week_seq#40] (59) ObjectHashAggregate -Input [1]: [d_week_seq#40] +Input [3]: [d_week_seq#40, propagatedFilter_1#94, propagatedFilter_2#95] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [buf#95] -Results [1]: [buf#96] +Functions [2]: [partial_bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0) FILTER (WHERE propagatedFilter_2#95), partial_bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0) FILTER (WHERE propagatedFilter_1#94)] +Aggregate Attributes [2]: [buf#96, buf#97] +Results [2]: [buf#98, buf#99] (60) Exchange -Input [1]: [buf#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +Input [2]: [buf#98, buf#99] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] (61) ObjectHashAggregate -Input [1]: [buf#96] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#97] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#97 AS bloomFilter#98] - -Subquery:2 Hosting operator id = 31 Hosting Expression = Subquery scalar-subquery#57, [id=#6] -ObjectHashAggregate (68) -+- Exchange (67) - +- ObjectHashAggregate (66) - +- * Project (65) - +- * Filter (64) - +- * ColumnarToRow (63) - +- Scan parquet spark_catalog.default.date_dim (62) - - -(62) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#78, d_week_seq#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1197), LessThanOrEqual(d_month_seq,1208), IsNotNull(d_week_seq)] -ReadSchema: struct - -(63) ColumnarToRow [codegen id : 1] -Input [2]: [d_month_seq#78, d_week_seq#79] - -(64) Filter [codegen id : 1] -Input [2]: [d_month_seq#78, d_week_seq#79] -Condition : (((isnotnull(d_month_seq#78) AND (d_month_seq#78 >= 1197)) AND (d_month_seq#78 <= 1208)) AND isnotnull(d_week_seq#79)) - -(65) Project [codegen id : 1] -Output [1]: [d_week_seq#79] -Input [2]: [d_month_seq#78, d_week_seq#79] - -(66) ObjectHashAggregate -Input [1]: [d_week_seq#79] +Input [2]: [buf#98, buf#99] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#79, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [buf#99] -Results [1]: [buf#100] +Functions [2]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0), bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)] +Aggregate Attributes [2]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#100, bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#101] +Results [2]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#100 AS bloomFilter#102, bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#101 AS bloomFilter#103] -(67) Exchange -Input [1]: [buf#100] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(62) Project [codegen id : 2] +Output [1]: [named_struct(bloomFilter, bloomFilter#102, bloomFilter, bloomFilter#103) AS mergedValue#104] +Input [2]: [bloomFilter#102, bloomFilter#103] -(68) ObjectHashAggregate -Input [1]: [buf#100] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#79, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#79, 42), 335, 8990, 0, 0)#101] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#79, 42), 335, 8990, 0, 0)#101 AS bloomFilter#102] +Subquery:2 Hosting operator id = 31 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#1] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt index 534396577ab9d..536bb9fb8e9ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt @@ -22,15 +22,18 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s WholeStageCodegen (1) Filter [d_date_sk,d_week_seq] Subquery #1 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 335, 8990, 0, 0),bloomFilter,buf] - Exchange #3 - ObjectHashAggregate [d_week_seq] [buf,buf] - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_month_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + WholeStageCodegen (2) + Project [bloomFilter,bloomFilter] + InputAdapter + ObjectHashAggregate [buf,buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 335, 8990, 0, 0),bloom_filter_agg(xxhash64(d_week_seq, 42), 335, 8990, 0, 0),bloomFilter,bloomFilter,buf,buf] + Exchange #3 + ObjectHashAggregate [d_week_seq,propagatedFilter_1,propagatedFilter_2] [buf,buf,buf,buf] + WholeStageCodegen (1) + Project [d_week_seq,d_month_seq] + Filter [d_month_seq,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] @@ -71,28 +74,19 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s BroadcastExchange #8 WholeStageCodegen (5) Filter [d_date_sk,d_week_seq] - Subquery #2 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 335, 8990, 0, 0),bloomFilter,buf] - Exchange #9 - ObjectHashAggregate [d_week_seq] [buf,buf] - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_month_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + ReusedSubquery [mergedValue] #1 ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (7) Filter [s_store_sk,s_store_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter - BroadcastExchange #11 + BroadcastExchange #10 WholeStageCodegen (8) Project [d_week_seq] Filter [d_month_seq,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt index b0f272919e6c9..fd8848872feb4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt @@ -84,7 +84,7 @@ Input [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] (6) Filter [codegen id : 1] Input [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] -Condition : ((isnotnull(d_date_sk#5) AND isnotnull(d_week_seq#6)) AND might_contain(Subquery scalar-subquery#8, [id=#1], xxhash64(d_week_seq#6, 42))) +Condition : ((isnotnull(d_date_sk#5) AND isnotnull(d_week_seq#6)) AND might_contain(Subquery scalar-subquery#8, [id=#1].bloomFilter, xxhash64(d_week_seq#6, 42))) (7) BroadcastExchange Input [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] @@ -205,11 +205,11 @@ Input [3]: [d_date_sk#58, d_week_seq#59, d_day_name#60] (32) Filter [codegen id : 7] Input [3]: [d_date_sk#58, d_week_seq#59, d_day_name#60] -Condition : ((isnotnull(d_date_sk#58) AND isnotnull(d_week_seq#59)) AND might_contain(Subquery scalar-subquery#61, [id=#5], xxhash64(d_week_seq#59, 42))) +Condition : ((isnotnull(d_date_sk#58) AND isnotnull(d_week_seq#59)) AND might_contain(ReusedSubquery Subquery scalar-subquery#8, [id=#1].bloomFilter, xxhash64(d_week_seq#59, 42))) (33) BroadcastExchange Input [3]: [d_date_sk#58, d_week_seq#59, d_day_name#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [sold_date_sk#56] @@ -222,31 +222,31 @@ Output [3]: [sales_price#57, d_week_seq#59, d_day_name#60] Input [5]: [sold_date_sk#56, sales_price#57, d_date_sk#58, d_week_seq#59, d_day_name#60] (36) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#62, cs_sold_date_sk#63] +Output [2]: [cs_ext_sales_price#61, cs_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#63)] +PartitionFilters: [isnotnull(cs_sold_date_sk#62)] ReadSchema: struct (37) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ext_sales_price#62, cs_sold_date_sk#63] +Input [2]: [cs_ext_sales_price#61, cs_sold_date_sk#62] (38) Project [codegen id : 10] -Output [2]: [cs_sold_date_sk#63 AS sold_date_sk#64, cs_ext_sales_price#62 AS sales_price#65] -Input [2]: [cs_ext_sales_price#62, cs_sold_date_sk#63] +Output [2]: [cs_sold_date_sk#62 AS sold_date_sk#63, cs_ext_sales_price#61 AS sales_price#64] +Input [2]: [cs_ext_sales_price#61, cs_sold_date_sk#62] (39) ReusedExchange [Reuses operator id: 33] -Output [3]: [d_date_sk#66, d_week_seq#67, d_day_name#68] +Output [3]: [d_date_sk#65, d_week_seq#66, d_day_name#67] (40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [sold_date_sk#64] -Right keys [1]: [d_date_sk#66] +Left keys [1]: [sold_date_sk#63] +Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None (41) Project [codegen id : 10] -Output [3]: [sales_price#65, d_week_seq#67, d_day_name#68] -Input [5]: [sold_date_sk#64, sales_price#65, d_date_sk#66, d_week_seq#67, d_day_name#68] +Output [3]: [sales_price#64, d_week_seq#66, d_day_name#67] +Input [5]: [sold_date_sk#63, sales_price#64, d_date_sk#65, d_week_seq#66, d_day_name#67] (42) Union @@ -254,91 +254,92 @@ Input [5]: [sold_date_sk#64, sales_price#65, d_date_sk#66, d_week_seq#67, d_day_ Input [3]: [sales_price#57, d_week_seq#59, d_day_name#60] Keys [1]: [d_week_seq#59] Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Sunday ) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Monday ) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Tuesday ) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Wednesday) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Thursday ) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Friday ) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Saturday ) THEN sales_price#57 END))] -Aggregate Attributes [7]: [sum#69, sum#70, sum#71, sum#72, sum#73, sum#74, sum#75] -Results [8]: [d_week_seq#59, sum#76, sum#77, sum#78, sum#79, sum#80, sum#81, sum#82] +Aggregate Attributes [7]: [sum#68, sum#69, sum#70, sum#71, sum#72, sum#73, sum#74] +Results [8]: [d_week_seq#59, sum#75, sum#76, sum#77, sum#78, sum#79, sum#80, sum#81] (44) Exchange -Input [8]: [d_week_seq#59, sum#76, sum#77, sum#78, sum#79, sum#80, sum#81, sum#82] -Arguments: hashpartitioning(d_week_seq#59, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [8]: [d_week_seq#59, sum#75, sum#76, sum#77, sum#78, sum#79, sum#80, sum#81] +Arguments: hashpartitioning(d_week_seq#59, 5), ENSURE_REQUIREMENTS, [plan_id=6] (45) HashAggregate [codegen id : 13] -Input [8]: [d_week_seq#59, sum#76, sum#77, sum#78, sum#79, sum#80, sum#81, sum#82] +Input [8]: [d_week_seq#59, sum#75, sum#76, sum#77, sum#78, sum#79, sum#80, sum#81] Keys [1]: [d_week_seq#59] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#60 = Sunday ) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Monday ) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Tuesday ) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Wednesday) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Thursday ) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Friday ) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Saturday ) THEN sales_price#57 END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#60 = Sunday ) THEN sales_price#57 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Monday ) THEN sales_price#57 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Tuesday ) THEN sales_price#57 END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Wednesday) THEN sales_price#57 END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Thursday ) THEN sales_price#57 END))#34, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Friday ) THEN sales_price#57 END))#35, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Saturday ) THEN sales_price#57 END))#36] -Results [8]: [d_week_seq#59, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Sunday ) THEN sales_price#57 END))#30,17,2) AS sun_sales#83, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Monday ) THEN sales_price#57 END))#31,17,2) AS mon_sales#84, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Tuesday ) THEN sales_price#57 END))#32,17,2) AS tue_sales#85, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Wednesday) THEN sales_price#57 END))#33,17,2) AS wed_sales#86, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Thursday ) THEN sales_price#57 END))#34,17,2) AS thu_sales#87, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Friday ) THEN sales_price#57 END))#35,17,2) AS fri_sales#88, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Saturday ) THEN sales_price#57 END))#36,17,2) AS sat_sales#89] +Results [8]: [d_week_seq#59, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Sunday ) THEN sales_price#57 END))#30,17,2) AS sun_sales#82, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Monday ) THEN sales_price#57 END))#31,17,2) AS mon_sales#83, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Tuesday ) THEN sales_price#57 END))#32,17,2) AS tue_sales#84, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Wednesday) THEN sales_price#57 END))#33,17,2) AS wed_sales#85, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Thursday ) THEN sales_price#57 END))#34,17,2) AS thu_sales#86, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Friday ) THEN sales_price#57 END))#35,17,2) AS fri_sales#87, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Saturday ) THEN sales_price#57 END))#36,17,2) AS sat_sales#88] (46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#90, d_year#91] +Output [2]: [d_week_seq#89, d_year#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct (47) ColumnarToRow [codegen id : 12] -Input [2]: [d_week_seq#90, d_year#91] +Input [2]: [d_week_seq#89, d_year#90] (48) Filter [codegen id : 12] -Input [2]: [d_week_seq#90, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2002)) AND isnotnull(d_week_seq#90)) +Input [2]: [d_week_seq#89, d_year#90] +Condition : ((isnotnull(d_year#90) AND (d_year#90 = 2002)) AND isnotnull(d_week_seq#89)) (49) Project [codegen id : 12] -Output [1]: [d_week_seq#90] -Input [2]: [d_week_seq#90, d_year#91] +Output [1]: [d_week_seq#89] +Input [2]: [d_week_seq#89, d_year#90] (50) BroadcastExchange -Input [1]: [d_week_seq#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Input [1]: [d_week_seq#89] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] (51) BroadcastHashJoin [codegen id : 13] Left keys [1]: [d_week_seq#59] -Right keys [1]: [d_week_seq#90] +Right keys [1]: [d_week_seq#89] Join type: Inner Join condition: None (52) Project [codegen id : 13] -Output [8]: [d_week_seq#59 AS d_week_seq2#92, sun_sales#83 AS sun_sales2#93, mon_sales#84 AS mon_sales2#94, tue_sales#85 AS tue_sales2#95, wed_sales#86 AS wed_sales2#96, thu_sales#87 AS thu_sales2#97, fri_sales#88 AS fri_sales2#98, sat_sales#89 AS sat_sales2#99] -Input [9]: [d_week_seq#59, sun_sales#83, mon_sales#84, tue_sales#85, wed_sales#86, thu_sales#87, fri_sales#88, sat_sales#89, d_week_seq#90] +Output [8]: [d_week_seq#59 AS d_week_seq2#91, sun_sales#82 AS sun_sales2#92, mon_sales#83 AS mon_sales2#93, tue_sales#84 AS tue_sales2#94, wed_sales#85 AS wed_sales2#95, thu_sales#86 AS thu_sales2#96, fri_sales#87 AS fri_sales2#97, sat_sales#88 AS sat_sales2#98] +Input [9]: [d_week_seq#59, sun_sales#82, mon_sales#83, tue_sales#84, wed_sales#85, thu_sales#86, fri_sales#87, sat_sales#88, d_week_seq#89] (53) BroadcastExchange -Input [8]: [d_week_seq2#92, sun_sales2#93, mon_sales2#94, tue_sales2#95, wed_sales2#96, thu_sales2#97, fri_sales2#98, sat_sales2#99] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=9] +Input [8]: [d_week_seq2#91, sun_sales2#92, mon_sales2#93, tue_sales2#94, wed_sales2#95, thu_sales2#96, fri_sales2#97, sat_sales2#98] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=8] (54) BroadcastHashJoin [codegen id : 14] Left keys [1]: [d_week_seq1#46] -Right keys [1]: [(d_week_seq2#92 - 53)] +Right keys [1]: [(d_week_seq2#91 - 53)] Join type: Inner Join condition: None (55) Project [codegen id : 14] -Output [8]: [d_week_seq1#46, round((sun_sales1#47 / sun_sales2#93), 2) AS round((sun_sales1 / sun_sales2), 2)#100, round((mon_sales1#48 / mon_sales2#94), 2) AS round((mon_sales1 / mon_sales2), 2)#101, round((tue_sales1#49 / tue_sales2#95), 2) AS round((tue_sales1 / tue_sales2), 2)#102, round((wed_sales1#50 / wed_sales2#96), 2) AS round((wed_sales1 / wed_sales2), 2)#103, round((thu_sales1#51 / thu_sales2#97), 2) AS round((thu_sales1 / thu_sales2), 2)#104, round((fri_sales1#52 / fri_sales2#98), 2) AS round((fri_sales1 / fri_sales2), 2)#105, round((sat_sales1#53 / sat_sales2#99), 2) AS round((sat_sales1 / sat_sales2), 2)#106] -Input [16]: [d_week_seq1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#92, sun_sales2#93, mon_sales2#94, tue_sales2#95, wed_sales2#96, thu_sales2#97, fri_sales2#98, sat_sales2#99] +Output [8]: [d_week_seq1#46, round((sun_sales1#47 / sun_sales2#92), 2) AS round((sun_sales1 / sun_sales2), 2)#99, round((mon_sales1#48 / mon_sales2#93), 2) AS round((mon_sales1 / mon_sales2), 2)#100, round((tue_sales1#49 / tue_sales2#94), 2) AS round((tue_sales1 / tue_sales2), 2)#101, round((wed_sales1#50 / wed_sales2#95), 2) AS round((wed_sales1 / wed_sales2), 2)#102, round((thu_sales1#51 / thu_sales2#96), 2) AS round((thu_sales1 / thu_sales2), 2)#103, round((fri_sales1#52 / fri_sales2#97), 2) AS round((fri_sales1 / fri_sales2), 2)#104, round((sat_sales1#53 / sat_sales2#98), 2) AS round((sat_sales1 / sat_sales2), 2)#105] +Input [16]: [d_week_seq1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#91, sun_sales2#92, mon_sales2#93, tue_sales2#94, wed_sales2#95, thu_sales2#96, fri_sales2#97, sat_sales2#98] (56) Exchange -Input [8]: [d_week_seq1#46, round((sun_sales1 / sun_sales2), 2)#100, round((mon_sales1 / mon_sales2), 2)#101, round((tue_sales1 / tue_sales2), 2)#102, round((wed_sales1 / wed_sales2), 2)#103, round((thu_sales1 / thu_sales2), 2)#104, round((fri_sales1 / fri_sales2), 2)#105, round((sat_sales1 / sat_sales2), 2)#106] -Arguments: rangepartitioning(d_week_seq1#46 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [8]: [d_week_seq1#46, round((sun_sales1 / sun_sales2), 2)#99, round((mon_sales1 / mon_sales2), 2)#100, round((tue_sales1 / tue_sales2), 2)#101, round((wed_sales1 / wed_sales2), 2)#102, round((thu_sales1 / thu_sales2), 2)#103, round((fri_sales1 / fri_sales2), 2)#104, round((sat_sales1 / sat_sales2), 2)#105] +Arguments: rangepartitioning(d_week_seq1#46 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=9] (57) Sort [codegen id : 15] -Input [8]: [d_week_seq1#46, round((sun_sales1 / sun_sales2), 2)#100, round((mon_sales1 / mon_sales2), 2)#101, round((tue_sales1 / tue_sales2), 2)#102, round((wed_sales1 / wed_sales2), 2)#103, round((thu_sales1 / thu_sales2), 2)#104, round((fri_sales1 / fri_sales2), 2)#105, round((sat_sales1 / sat_sales2), 2)#106] +Input [8]: [d_week_seq1#46, round((sun_sales1 / sun_sales2), 2)#99, round((mon_sales1 / mon_sales2), 2)#100, round((tue_sales1 / tue_sales2), 2)#101, round((wed_sales1 / wed_sales2), 2)#102, round((thu_sales1 / thu_sales2), 2)#103, round((fri_sales1 / fri_sales2), 2)#104, round((sat_sales1 / sat_sales2), 2)#105] Arguments: [d_week_seq1#46 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery#8, [id=#1] -ObjectHashAggregate (64) -+- Exchange (63) - +- ObjectHashAggregate (62) - +- * Project (61) - +- * Filter (60) - +- * ColumnarToRow (59) - +- Scan parquet spark_catalog.default.date_dim (58) +* Project (65) ++- ObjectHashAggregate (64) + +- Exchange (63) + +- ObjectHashAggregate (62) + +- * Project (61) + +- * Filter (60) + +- * ColumnarToRow (59) + +- Scan parquet spark_catalog.default.date_dim (58) (58) Scan parquet spark_catalog.default.date_dim Output [2]: [d_week_seq#44, d_year#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_week_seq), Or(EqualTo(d_year,2002),EqualTo(d_year,2001))] ReadSchema: struct (59) ColumnarToRow [codegen id : 1] @@ -346,74 +347,34 @@ Input [2]: [d_week_seq#44, d_year#45] (60) Filter [codegen id : 1] Input [2]: [d_week_seq#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2001)) AND isnotnull(d_week_seq#44)) +Condition : ((isnotnull(d_year#45) AND isnotnull(d_week_seq#44)) AND ((d_year#45 = 2002) OR (d_year#45 = 2001))) (61) Project [codegen id : 1] -Output [1]: [d_week_seq#44] +Output [3]: [d_week_seq#44, ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_week_seq#44)) AS propagatedFilter_1#106, ((isnotnull(d_year#45) AND (d_year#45 = 2001)) AND isnotnull(d_week_seq#44)) AS propagatedFilter_2#107] Input [2]: [d_week_seq#44, d_year#45] (62) ObjectHashAggregate -Input [1]: [d_week_seq#44] +Input [3]: [d_week_seq#44, propagatedFilter_1#106, propagatedFilter_2#107] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [buf#107] -Results [1]: [buf#108] +Functions [2]: [partial_bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0) FILTER (WHERE propagatedFilter_2#107), partial_bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0) FILTER (WHERE propagatedFilter_1#106)] +Aggregate Attributes [2]: [buf#108, buf#109] +Results [2]: [buf#110, buf#111] (63) Exchange -Input [1]: [buf#108] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] +Input [2]: [buf#110, buf#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] (64) ObjectHashAggregate -Input [1]: [buf#108] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)#109] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)#109 AS bloomFilter#110] - -Subquery:2 Hosting operator id = 32 Hosting Expression = Subquery scalar-subquery#61, [id=#5] -ObjectHashAggregate (71) -+- Exchange (70) - +- ObjectHashAggregate (69) - +- * Project (68) - +- * Filter (67) - +- * ColumnarToRow (66) - +- Scan parquet spark_catalog.default.date_dim (65) - - -(65) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#90, d_year#91] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(66) ColumnarToRow [codegen id : 1] -Input [2]: [d_week_seq#90, d_year#91] - -(67) Filter [codegen id : 1] -Input [2]: [d_week_seq#90, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2002)) AND isnotnull(d_week_seq#90)) - -(68) Project [codegen id : 1] -Output [1]: [d_week_seq#90] -Input [2]: [d_week_seq#90, d_year#91] - -(69) ObjectHashAggregate -Input [1]: [d_week_seq#90] +Input [2]: [buf#110, buf#111] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#90, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [buf#111] -Results [1]: [buf#112] +Functions [2]: [bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0), bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)] +Aggregate Attributes [2]: [bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)#112, bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)#113] +Results [2]: [bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)#112 AS bloomFilter#114, bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)#113 AS bloomFilter#115] -(70) Exchange -Input [1]: [buf#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(65) Project [codegen id : 2] +Output [1]: [named_struct(bloomFilter, bloomFilter#114, bloomFilter, bloomFilter#115) AS mergedValue#116] +Input [2]: [bloomFilter#114, bloomFilter#115] -(71) ObjectHashAggregate -Input [1]: [buf#112] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#90, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#90, 42), 362, 9656, 0, 0)#113] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#90, 42), 362, 9656, 0, 0)#113 AS bloomFilter#114] +Subquery:2 Hosting operator id = 32 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#1] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt index e4a8fb4c2f156..9dcda91391e27 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt @@ -26,15 +26,18 @@ WholeStageCodegen (15) WholeStageCodegen (1) Filter [d_date_sk,d_week_seq] Subquery #1 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 362, 9656, 0, 0),bloomFilter,buf] - Exchange #4 - ObjectHashAggregate [d_week_seq] [buf,buf] - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + WholeStageCodegen (2) + Project [bloomFilter,bloomFilter] + InputAdapter + ObjectHashAggregate [buf,buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 362, 9656, 0, 0),bloom_filter_agg(xxhash64(d_week_seq, 42), 362, 9656, 0, 0),bloomFilter,bloomFilter,buf,buf] + Exchange #4 + ObjectHashAggregate [d_week_seq,propagatedFilter_1,propagatedFilter_2] [buf,buf,buf,buf] + WholeStageCodegen (1) + Project [d_week_seq,d_year] + Filter [d_year,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] @@ -78,16 +81,7 @@ WholeStageCodegen (15) BroadcastExchange #8 WholeStageCodegen (7) Filter [d_date_sk,d_week_seq] - Subquery #2 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 362, 9656, 0, 0),bloomFilter,buf] - Exchange #9 - ObjectHashAggregate [d_week_seq] [buf,buf] - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + ReusedSubquery [mergedValue] #1 ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] @@ -101,7 +95,7 @@ WholeStageCodegen (15) InputAdapter ReusedExchange [d_date_sk,d_week_seq,d_day_name] #8 InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (12) Project [d_week_seq] Filter [d_year,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt index bfc684b1488d6..e9970ea0dd88d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt @@ -1,437 +1,221 @@ == Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (70) -:- * BroadcastNestedLoopJoin Inner BuildRight (58) -: :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (34) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (22) -: : : : :- * HashAggregate (10) -: : : : : +- Exchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- Exchange (6) -: : : : : +- * HashAggregate (5) -: : : : : +- * Project (4) -: : : : : +- * Filter (3) -: : : : : +- * ColumnarToRow (2) -: : : : : +- Scan parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (21) -: : : : +- * HashAggregate (20) -: : : : +- Exchange (19) -: : : : +- * HashAggregate (18) -: : : : +- * HashAggregate (17) -: : : : +- Exchange (16) -: : : : +- * HashAggregate (15) -: : : : +- * Project (14) -: : : : +- * Filter (13) -: : : : +- * ColumnarToRow (12) -: : : : +- Scan parquet spark_catalog.default.store_sales (11) -: : : +- BroadcastExchange (33) -: : : +- * HashAggregate (32) -: : : +- Exchange (31) -: : : +- * HashAggregate (30) -: : : +- * HashAggregate (29) -: : : +- Exchange (28) -: : : +- * HashAggregate (27) -: : : +- * Project (26) -: : : +- * Filter (25) -: : : +- * ColumnarToRow (24) -: : : +- Scan parquet spark_catalog.default.store_sales (23) -: : +- BroadcastExchange (45) -: : +- * HashAggregate (44) -: : +- Exchange (43) -: : +- * HashAggregate (42) -: : +- * HashAggregate (41) -: : +- Exchange (40) -: : +- * HashAggregate (39) -: : +- * Project (38) -: : +- * Filter (37) -: : +- * ColumnarToRow (36) -: : +- Scan parquet spark_catalog.default.store_sales (35) -: +- BroadcastExchange (57) -: +- * HashAggregate (56) -: +- Exchange (55) -: +- * HashAggregate (54) -: +- * HashAggregate (53) -: +- Exchange (52) -: +- * HashAggregate (51) -: +- * Project (50) -: +- * Filter (49) -: +- * ColumnarToRow (48) -: +- Scan parquet spark_catalog.default.store_sales (47) -+- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet spark_catalog.default.store_sales (59) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct +* BroadcastNestedLoopJoin Inner BuildRight (22) +:- * BroadcastNestedLoopJoin Inner BuildRight (18) +: :- * BroadcastNestedLoopJoin Inner BuildRight (14) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (10) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (6) +: : : : :- * Project (2) +: : : : : +- * Scan OneRowRelation (1) +: : : : +- BroadcastExchange (5) +: : : : +- * Project (4) +: : : : +- * Scan OneRowRelation (3) +: : : +- BroadcastExchange (9) +: : : +- * Project (8) +: : : +- * Scan OneRowRelation (7) +: : +- BroadcastExchange (13) +: : +- * Project (12) +: : +- * Scan OneRowRelation (11) +: +- BroadcastExchange (17) +: +- * Project (16) +: +- * Scan OneRowRelation (15) ++- BroadcastExchange (21) + +- * Project (20) + +- * Scan OneRowRelation (19) + + +(1) Scan OneRowRelation [codegen id : 6] +Output: [] + +(2) Project [codegen id : 6] +Output [3]: [Subquery scalar-subquery#1, [id=#1].B1_LP AS B1_LP#2, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B1_CNT AS B1_CNT#3, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B1_CNTD AS B1_CNTD#4] +Input: [] + +(3) Scan OneRowRelation [codegen id : 1] +Output: [] -(2) ColumnarToRow [codegen id : 1] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +(4) Project [codegen id : 1] +Output [3]: [ReusedSubquery Subquery scalar-subquery#1, [id=#1].B2_LP AS B2_LP#5, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B2_CNT AS B2_CNT#6, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B2_CNTD AS B2_CNTD#7] +Input: [] -(3) Filter [codegen id : 1] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) +(5) BroadcastExchange +Input [3]: [B2_LP#5, B2_CNT#6, B2_CNTD#7] +Arguments: IdentityBroadcastMode, [plan_id=2] -(4) Project [codegen id : 1] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] - -(5) HashAggregate [codegen id : 1] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7] -Results [4]: [ss_list_price#3, sum#8, count#9, count#10] - -(6) Exchange -Input [4]: [ss_list_price#3, sum#8, count#9, count#10] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(7) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#3, sum#8, count#9, count#10] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7] -Results [4]: [ss_list_price#3, sum#8, count#9, count#10] - -(8) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#3, sum#8, count#9, count#10] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7, count(ss_list_price#3)#11] -Results [4]: [sum#8, count#9, count#10, count#12] +(6) BroadcastNestedLoopJoin [codegen id : 6] +Join type: Inner +Join condition: None -(9) Exchange -Input [4]: [sum#8, count#9, count#10, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +(7) Scan OneRowRelation [codegen id : 2] +Output: [] -(10) HashAggregate [codegen id : 18] -Input [4]: [sum#8, count#9, count#10, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7, count(ss_list_price#3)#11] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#13, count(ss_list_price#3)#7 AS B1_CNT#14, count(ss_list_price#3)#11 AS B1_CNTD#15] +(8) Project [codegen id : 2] +Output [3]: [ReusedSubquery Subquery scalar-subquery#1, [id=#1].B3_LP AS B3_LP#8, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B3_CNT AS B3_CNT#9, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B3_CNTD AS B3_CNTD#10] +Input: [] -(11) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct +(9) BroadcastExchange +Input [3]: [B3_LP#8, B3_CNT#9, B3_CNTD#10] +Arguments: IdentityBroadcastMode, [plan_id=3] -(12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] - -(13) Filter [codegen id : 3] -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) Project [codegen id : 3] -Output [1]: [ss_list_price#18] -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] - -(15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22] -Results [4]: [ss_list_price#18, sum#23, count#24, count#25] - -(16) Exchange -Input [4]: [ss_list_price#18, sum#23, count#24, count#25] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#18, sum#23, count#24, count#25] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22] -Results [4]: [ss_list_price#18, sum#23, count#24, count#25] - -(18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#18, sum#23, count#24, count#25] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22, count(ss_list_price#18)#26] -Results [4]: [sum#23, count#24, count#25, count#27] +(10) BroadcastNestedLoopJoin [codegen id : 6] +Join type: Inner +Join condition: None -(19) Exchange -Input [4]: [sum#23, count#24, count#25, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(11) Scan OneRowRelation [codegen id : 3] +Output: [] -(20) HashAggregate [codegen id : 5] -Input [4]: [sum#23, count#24, count#25, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22, count(ss_list_price#18)#26] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#18))#21 / 100.0) as decimal(11,6)) AS B2_LP#28, count(ss_list_price#18)#22 AS B2_CNT#29, count(ss_list_price#18)#26 AS B2_CNTD#30] +(12) Project [codegen id : 3] +Output [3]: [ReusedSubquery Subquery scalar-subquery#1, [id=#1].B4_LP AS B4_LP#11, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B4_CNT AS B4_CNT#12, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B4_CNTD AS B4_CNTD#13] +Input: [] -(21) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] +(13) BroadcastExchange +Input [3]: [B4_LP#11, B4_CNT#12, B4_CNTD#13] +Arguments: IdentityBroadcastMode, [plan_id=4] -(22) BroadcastNestedLoopJoin [codegen id : 18] +(14) BroadcastNestedLoopJoin [codegen id : 6] Join type: Inner Join condition: None -(23) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct +(15) Scan OneRowRelation [codegen id : 4] +Output: [] -(24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] - -(25) Filter [codegen id : 6] -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(26) Project [codegen id : 6] -Output [1]: [ss_list_price#33] -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] - -(27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37] -Results [4]: [ss_list_price#33, sum#38, count#39, count#40] - -(28) Exchange -Input [4]: [ss_list_price#33, sum#38, count#39, count#40] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#33, sum#38, count#39, count#40] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37] -Results [4]: [ss_list_price#33, sum#38, count#39, count#40] - -(30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#33, sum#38, count#39, count#40] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37, count(ss_list_price#33)#41] -Results [4]: [sum#38, count#39, count#40, count#42] +(16) Project [codegen id : 4] +Output [3]: [ReusedSubquery Subquery scalar-subquery#1, [id=#1].B5_LP AS B5_LP#14, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B5_CNT AS B5_CNT#15, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B5_CNTD AS B5_CNTD#16] +Input: [] -(31) Exchange -Input [4]: [sum#38, count#39, count#40, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(17) BroadcastExchange +Input [3]: [B5_LP#14, B5_CNT#15, B5_CNTD#16] +Arguments: IdentityBroadcastMode, [plan_id=5] -(32) HashAggregate [codegen id : 8] -Input [4]: [sum#38, count#39, count#40, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37, count(ss_list_price#33)#41] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#33))#36 / 100.0) as decimal(11,6)) AS B3_LP#43, count(ss_list_price#33)#37 AS B3_CNT#44, count(ss_list_price#33)#41 AS B3_CNTD#45] +(18) BroadcastNestedLoopJoin [codegen id : 6] +Join type: Inner +Join condition: None + +(19) Scan OneRowRelation [codegen id : 5] +Output: [] -(33) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] +(20) Project [codegen id : 5] +Output [3]: [ReusedSubquery Subquery scalar-subquery#1, [id=#1].B6_LP AS B6_LP#17, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B6_CNT AS B6_CNT#18, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B6_CNTD AS B6_CNTD#19] +Input: [] -(34) BroadcastNestedLoopJoin [codegen id : 18] +(21) BroadcastExchange +Input [3]: [B6_LP#17, B6_CNT#18, B6_CNTD#19] +Arguments: IdentityBroadcastMode, [plan_id=6] + +(22) BroadcastNestedLoopJoin [codegen id : 6] Join type: Inner Join condition: None -(35) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] +===== Subqueries ===== + +Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery scalar-subquery#1, [id=#1] +* Project (34) ++- * HashAggregate (33) + +- Exchange (32) + +- * HashAggregate (31) + +- * HashAggregate (30) + +- Exchange (29) + +- * HashAggregate (28) + +- * Expand (27) + +- * Project (26) + +- * Filter (25) + +- * ColumnarToRow (24) + +- Scan parquet spark_catalog.default.store_sales (23) + + +(23) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_quantity#20, ss_wholesale_cost#21, ss_list_price#22, ss_coupon_amt#23, ss_sold_date_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] +PushedFilters: [IsNotNull(ss_quantity), Or(Or(Or(Or(Or(And(And(GreaterThanOrEqual(ss_quantity,6),LessThanOrEqual(ss_quantity,10)),Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))),And(And(GreaterThanOrEqual(ss_quantity,0),LessThanOrEqual(ss_quantity,5)),Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00))))),And(And(GreaterThanOrEqual(ss_quantity,11),LessThanOrEqual(ss_quantity,15)),Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00))))),And(And(GreaterThanOrEqual(ss_quantity,16),LessThanOrEqual(ss_quantity,20)),Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00))))),And(And(GreaterThanOrEqual(ss_quantity,21),LessThanOrEqual(ss_quantity,25)),Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00))))),And(And(GreaterThanOrEqual(ss_quantity,26),LessThanOrEqual(ss_quantity,30)),Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))))] ReadSchema: struct -(36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] - -(37) Filter [codegen id : 9] -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(38) Project [codegen id : 9] -Output [1]: [ss_list_price#48] -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] - -(39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52] -Results [4]: [ss_list_price#48, sum#53, count#54, count#55] - -(40) Exchange -Input [4]: [ss_list_price#48, sum#53, count#54, count#55] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#48, sum#53, count#54, count#55] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52] -Results [4]: [ss_list_price#48, sum#53, count#54, count#55] - -(42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#48, sum#53, count#54, count#55] +(24) ColumnarToRow [codegen id : 1] +Input [5]: [ss_quantity#20, ss_wholesale_cost#21, ss_list_price#22, ss_coupon_amt#23, ss_sold_date_sk#24] + +(25) Filter [codegen id : 1] +Input [5]: [ss_quantity#20, ss_wholesale_cost#21, ss_list_price#22, ss_coupon_amt#23, ss_sold_date_sk#24] +Condition : (isnotnull(ss_quantity#20) AND ((((((((ss_quantity#20 >= 6) AND (ss_quantity#20 <= 10)) AND ((((ss_list_price#22 >= 90.00) AND (ss_list_price#22 <= 100.00)) OR ((ss_coupon_amt#23 >= 2323.00) AND (ss_coupon_amt#23 <= 3323.00))) OR ((ss_wholesale_cost#21 >= 31.00) AND (ss_wholesale_cost#21 <= 51.00)))) OR (((ss_quantity#20 >= 0) AND (ss_quantity#20 <= 5)) AND ((((ss_list_price#22 >= 8.00) AND (ss_list_price#22 <= 18.00)) OR ((ss_coupon_amt#23 >= 459.00) AND (ss_coupon_amt#23 <= 1459.00))) OR ((ss_wholesale_cost#21 >= 57.00) AND (ss_wholesale_cost#21 <= 77.00))))) OR (((ss_quantity#20 >= 11) AND (ss_quantity#20 <= 15)) AND ((((ss_list_price#22 >= 142.00) AND (ss_list_price#22 <= 152.00)) OR ((ss_coupon_amt#23 >= 12214.00) AND (ss_coupon_amt#23 <= 13214.00))) OR ((ss_wholesale_cost#21 >= 79.00) AND (ss_wholesale_cost#21 <= 99.00))))) OR (((ss_quantity#20 >= 16) AND (ss_quantity#20 <= 20)) AND ((((ss_list_price#22 >= 135.00) AND (ss_list_price#22 <= 145.00)) OR ((ss_coupon_amt#23 >= 6071.00) AND (ss_coupon_amt#23 <= 7071.00))) OR ((ss_wholesale_cost#21 >= 38.00) AND (ss_wholesale_cost#21 <= 58.00))))) OR (((ss_quantity#20 >= 21) AND (ss_quantity#20 <= 25)) AND ((((ss_list_price#22 >= 122.00) AND (ss_list_price#22 <= 132.00)) OR ((ss_coupon_amt#23 >= 836.00) AND (ss_coupon_amt#23 <= 1836.00))) OR ((ss_wholesale_cost#21 >= 17.00) AND (ss_wholesale_cost#21 <= 37.00))))) OR (((ss_quantity#20 >= 26) AND (ss_quantity#20 <= 30)) AND ((((ss_list_price#22 >= 154.00) AND (ss_list_price#22 <= 164.00)) OR ((ss_coupon_amt#23 >= 7326.00) AND (ss_coupon_amt#23 <= 8326.00))) OR ((ss_wholesale_cost#21 >= 7.00) AND (ss_wholesale_cost#21 <= 27.00)))))) + +(26) Project [codegen id : 1] +Output [7]: [ss_list_price#22, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 6) AND (ss_quantity#20 <= 10)) AND ((((ss_list_price#22 >= 90.00) AND (ss_list_price#22 <= 100.00)) OR ((ss_coupon_amt#23 >= 2323.00) AND (ss_coupon_amt#23 <= 3323.00))) OR ((ss_wholesale_cost#21 >= 31.00) AND (ss_wholesale_cost#21 <= 51.00))))) AS propagatedFilter_1#25, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 0) AND (ss_quantity#20 <= 5)) AND ((((ss_list_price#22 >= 8.00) AND (ss_list_price#22 <= 18.00)) OR ((ss_coupon_amt#23 >= 459.00) AND (ss_coupon_amt#23 <= 1459.00))) OR ((ss_wholesale_cost#21 >= 57.00) AND (ss_wholesale_cost#21 <= 77.00))))) AS propagatedFilter_2#26, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 11) AND (ss_quantity#20 <= 15)) AND ((((ss_list_price#22 >= 142.00) AND (ss_list_price#22 <= 152.00)) OR ((ss_coupon_amt#23 >= 12214.00) AND (ss_coupon_amt#23 <= 13214.00))) OR ((ss_wholesale_cost#21 >= 79.00) AND (ss_wholesale_cost#21 <= 99.00))))) AS propagatedFilter_3#27, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 16) AND (ss_quantity#20 <= 20)) AND ((((ss_list_price#22 >= 135.00) AND (ss_list_price#22 <= 145.00)) OR ((ss_coupon_amt#23 >= 6071.00) AND (ss_coupon_amt#23 <= 7071.00))) OR ((ss_wholesale_cost#21 >= 38.00) AND (ss_wholesale_cost#21 <= 58.00))))) AS propagatedFilter_4#28, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 21) AND (ss_quantity#20 <= 25)) AND ((((ss_list_price#22 >= 122.00) AND (ss_list_price#22 <= 132.00)) OR ((ss_coupon_amt#23 >= 836.00) AND (ss_coupon_amt#23 <= 1836.00))) OR ((ss_wholesale_cost#21 >= 17.00) AND (ss_wholesale_cost#21 <= 37.00))))) AS propagatedFilter_5#29, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 26) AND (ss_quantity#20 <= 30)) AND ((((ss_list_price#22 >= 154.00) AND (ss_list_price#22 <= 164.00)) OR ((ss_coupon_amt#23 >= 7326.00) AND (ss_coupon_amt#23 <= 8326.00))) OR ((ss_wholesale_cost#21 >= 7.00) AND (ss_wholesale_cost#21 <= 27.00))))) AS propagatedFilter_6#30] +Input [5]: [ss_quantity#20, ss_wholesale_cost#21, ss_list_price#22, ss_coupon_amt#23, ss_sold_date_sk#24] + +(27) Expand [codegen id : 1] +Input [7]: [ss_list_price#22, propagatedFilter_1#25, propagatedFilter_2#26, propagatedFilter_3#27, propagatedFilter_4#28, propagatedFilter_5#29, propagatedFilter_6#30] +Arguments: [[null, 0, null, null, null, null, null, null, UnscaledValue(ss_list_price#22), ss_list_price#22, UnscaledValue(ss_list_price#22), ss_list_price#22, propagatedFilter_2#26, propagatedFilter_1#25, propagatedFilter_3#27, propagatedFilter_4#28, propagatedFilter_5#29, propagatedFilter_6#30], [ss_list_price#22, 1, propagatedFilter_2#26, propagatedFilter_1#25, propagatedFilter_3#27, propagatedFilter_4#28, propagatedFilter_5#29, propagatedFilter_6#30, null, null, null, null, null, null, null, null, null, null]], [spark_catalog.default.store_sales.ss_list_price#31, gid#32, propagatedFilter_2#33, propagatedFilter_1#34, propagatedFilter_3#35, propagatedFilter_4#36, propagatedFilter_5#37, propagatedFilter_6#38, unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39, spark_catalog.default.store_sales.ss_list_price#40, unscaledvalue(ss_list_price)#41, ss_list_price#42, propagatedFilter_2#43, propagatedFilter_1#44, propagatedFilter_3#45, propagatedFilter_4#46, propagatedFilter_5#47, propagatedFilter_6#48] + +(28) HashAggregate [codegen id : 1] +Input [18]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, propagatedFilter_2#33, propagatedFilter_1#34, propagatedFilter_3#35, propagatedFilter_4#36, propagatedFilter_5#37, propagatedFilter_6#38, unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39, spark_catalog.default.store_sales.ss_list_price#40, unscaledvalue(ss_list_price)#41, ss_list_price#42, propagatedFilter_2#43, propagatedFilter_1#44, propagatedFilter_3#45, propagatedFilter_4#46, propagatedFilter_5#47, propagatedFilter_6#48] +Keys [2]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32] +Functions [18]: [partial_max(propagatedFilter_2#33), partial_max(propagatedFilter_1#34), partial_max(propagatedFilter_3#35), partial_max(propagatedFilter_4#36), partial_max(propagatedFilter_5#37), partial_max(propagatedFilter_6#38), partial_avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39) FILTER (WHERE propagatedFilter_2#43), partial_count(spark_catalog.default.store_sales.ss_list_price#40) FILTER (WHERE propagatedFilter_2#43), partial_avg(unscaledvalue(ss_list_price)#41) FILTER (WHERE propagatedFilter_1#44), partial_count(ss_list_price#42) FILTER (WHERE propagatedFilter_1#44), partial_avg(unscaledvalue(ss_list_price)#41) FILTER (WHERE propagatedFilter_3#45), partial_count(ss_list_price#42) FILTER (WHERE propagatedFilter_3#45), partial_avg(unscaledvalue(ss_list_price)#41) FILTER (WHERE propagatedFilter_4#46), partial_count(ss_list_price#42) FILTER (WHERE propagatedFilter_4#46), partial_avg(unscaledvalue(ss_list_price)#41) FILTER (WHERE propagatedFilter_5#47), partial_count(ss_list_price#42) FILTER (WHERE propagatedFilter_5#47), partial_avg(unscaledvalue(ss_list_price)#41) FILTER (WHERE propagatedFilter_6#48), partial_count(ss_list_price#42) FILTER (WHERE propagatedFilter_6#48)] +Aggregate Attributes [24]: [max#49, max#50, max#51, max#52, max#53, max#54, sum#55, count#56, count#57, sum#58, count#59, count#60, sum#61, count#62, count#63, sum#64, count#65, count#66, sum#67, count#68, count#69, sum#70, count#71, count#72] +Results [26]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, max#73, max#74, max#75, max#76, max#77, max#78, sum#79, count#80, count#81, sum#82, count#83, count#84, sum#85, count#86, count#87, sum#88, count#89, count#90, sum#91, count#92, count#93, sum#94, count#95, count#96] + +(29) Exchange +Input [26]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, max#73, max#74, max#75, max#76, max#77, max#78, sum#79, count#80, count#81, sum#82, count#83, count#84, sum#85, count#86, count#87, sum#88, count#89, count#90, sum#91, count#92, count#93, sum#94, count#95, count#96] +Arguments: hashpartitioning(spark_catalog.default.store_sales.ss_list_price#31, gid#32, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(30) HashAggregate [codegen id : 2] +Input [26]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, max#73, max#74, max#75, max#76, max#77, max#78, sum#79, count#80, count#81, sum#82, count#83, count#84, sum#85, count#86, count#87, sum#88, count#89, count#90, sum#91, count#92, count#93, sum#94, count#95, count#96] +Keys [2]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32] +Functions [18]: [max(propagatedFilter_2#33), max(propagatedFilter_1#34), max(propagatedFilter_3#35), max(propagatedFilter_4#36), max(propagatedFilter_5#37), max(propagatedFilter_6#38), avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39), count(spark_catalog.default.store_sales.ss_list_price#40), avg(unscaledvalue(ss_list_price)#41), count(ss_list_price#42), avg(unscaledvalue(ss_list_price)#41), count(ss_list_price#42), avg(unscaledvalue(ss_list_price)#41), count(ss_list_price#42), avg(unscaledvalue(ss_list_price)#41), count(ss_list_price#42), avg(unscaledvalue(ss_list_price)#41), count(ss_list_price#42)] +Aggregate Attributes [18]: [max(propagatedFilter_2#33)#97, max(propagatedFilter_1#34)#98, max(propagatedFilter_3#35)#99, max(propagatedFilter_4#36)#100, max(propagatedFilter_5#37)#101, max(propagatedFilter_6#38)#102, avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39)#103, count(spark_catalog.default.store_sales.ss_list_price#40)#104, avg(unscaledvalue(ss_list_price)#41)#105, count(ss_list_price#42)#106, avg(unscaledvalue(ss_list_price)#41)#107, count(ss_list_price#42)#108, avg(unscaledvalue(ss_list_price)#41)#109, count(ss_list_price#42)#110, avg(unscaledvalue(ss_list_price)#41)#111, count(ss_list_price#42)#112, avg(unscaledvalue(ss_list_price)#41)#113, count(ss_list_price#42)#114] +Results [20]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, max(propagatedFilter_2#33)#97 AS propagatedFilter_2#115, max(propagatedFilter_1#34)#98 AS propagatedFilter_1#116, max(propagatedFilter_3#35)#99 AS propagatedFilter_3#117, max(propagatedFilter_4#36)#100 AS propagatedFilter_4#118, max(propagatedFilter_5#37)#101 AS propagatedFilter_5#119, max(propagatedFilter_6#38)#102 AS propagatedFilter_6#120, avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39)#103 AS avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121, count(spark_catalog.default.store_sales.ss_list_price#40)#104 AS count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122, avg(unscaledvalue(ss_list_price)#41)#105 AS avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123, count(ss_list_price#42)#106 AS count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124, avg(unscaledvalue(ss_list_price)#41)#107 AS avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125, count(ss_list_price#42)#108 AS count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126, avg(unscaledvalue(ss_list_price)#41)#109 AS avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127, count(ss_list_price#42)#110 AS count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128, avg(unscaledvalue(ss_list_price)#41)#111 AS avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129, count(ss_list_price#42)#112 AS count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130, avg(unscaledvalue(ss_list_price)#41)#113 AS avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131, count(ss_list_price#42)#114 AS count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132] + +(31) HashAggregate [codegen id : 2] +Input [20]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, propagatedFilter_2#115, propagatedFilter_1#116, propagatedFilter_3#117, propagatedFilter_4#118, propagatedFilter_5#119, propagatedFilter_6#120, avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121, count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122, avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123, count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124, avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125, count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126, avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127, count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128, avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129, count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130, avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131, count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52, count(ss_list_price#48)#56] -Results [4]: [sum#53, count#54, count#55, count#57] +Functions [18]: [partial_first(avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_2#115)), partial_first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_1#116)), partial_first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_3#117)), partial_first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_4#118)), partial_first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_5#119)), partial_first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_6#120))] +Aggregate Attributes [30]: [first#133, valueSet#134, first#135, valueSet#136, count#137, first#138, valueSet#139, first#140, valueSet#141, count#142, first#143, valueSet#144, first#145, valueSet#146, count#147, first#148, valueSet#149, first#150, valueSet#151, count#152, first#153, valueSet#154, first#155, valueSet#156, count#157, first#158, valueSet#159, first#160, valueSet#161, count#162] +Results [30]: [first#163, valueSet#164, first#165, valueSet#166, count#167, first#168, valueSet#169, first#170, valueSet#171, count#172, first#173, valueSet#174, first#175, valueSet#176, count#177, first#178, valueSet#179, first#180, valueSet#181, count#182, first#183, valueSet#184, first#185, valueSet#186, count#187, first#188, valueSet#189, first#190, valueSet#191, count#192] -(43) Exchange -Input [4]: [sum#53, count#54, count#55, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] +(32) Exchange +Input [30]: [first#163, valueSet#164, first#165, valueSet#166, count#167, first#168, valueSet#169, first#170, valueSet#171, count#172, first#173, valueSet#174, first#175, valueSet#176, count#177, first#178, valueSet#179, first#180, valueSet#181, count#182, first#183, valueSet#184, first#185, valueSet#186, count#187, first#188, valueSet#189, first#190, valueSet#191, count#192] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(44) HashAggregate [codegen id : 11] -Input [4]: [sum#53, count#54, count#55, count#57] +(33) HashAggregate [codegen id : 3] +Input [30]: [first#163, valueSet#164, first#165, valueSet#166, count#167, first#168, valueSet#169, first#170, valueSet#171, count#172, first#173, valueSet#174, first#175, valueSet#176, count#177, first#178, valueSet#179, first#180, valueSet#181, count#182, first#183, valueSet#184, first#185, valueSet#186, count#187, first#188, valueSet#189, first#190, valueSet#191, count#192] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52, count(ss_list_price#48)#56] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#48))#51 / 100.0) as decimal(11,6)) AS B4_LP#58, count(ss_list_price#48)#52 AS B4_CNT#59, count(ss_list_price#48)#56 AS B4_CNTD#60] +Functions [18]: [first(avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121, true), first(count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122, true), count(spark_catalog.default.store_sales.ss_list_price#31), first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123, true), first(count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124, true), count(spark_catalog.default.store_sales.ss_list_price#31), first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125, true), first(count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126, true), count(spark_catalog.default.store_sales.ss_list_price#31), first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127, true), first(count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128, true), count(spark_catalog.default.store_sales.ss_list_price#31), first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129, true), first(count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130, true), count(spark_catalog.default.store_sales.ss_list_price#31), first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131, true), first(count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132, true), count(spark_catalog.default.store_sales.ss_list_price#31)] +Aggregate Attributes [18]: [first(avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121) ignore nulls#193, first(count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122) ignore nulls#194, count(spark_catalog.default.store_sales.ss_list_price#31)#195, first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123) ignore nulls#196, first(count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124) ignore nulls#197, count(spark_catalog.default.store_sales.ss_list_price#31)#198, first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125) ignore nulls#199, first(count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126) ignore nulls#200, count(spark_catalog.default.store_sales.ss_list_price#31)#201, first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127) ignore nulls#202, first(count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128) ignore nulls#203, count(spark_catalog.default.store_sales.ss_list_price#31)#204, first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129) ignore nulls#205, first(count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130) ignore nulls#206, count(spark_catalog.default.store_sales.ss_list_price#31)#207, first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131) ignore nulls#208, first(count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132) ignore nulls#209, count(spark_catalog.default.store_sales.ss_list_price#31)#210] +Results [18]: [cast((first(avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121) ignore nulls#193 / 100.0) as decimal(11,6)) AS B1_LP#2, coalesce(first(count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122) ignore nulls#194, 0) AS B1_CNT#3, count(spark_catalog.default.store_sales.ss_list_price#31)#195 AS B1_CNTD#4, cast((first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123) ignore nulls#196 / 100.0) as decimal(11,6)) AS B2_LP#5, coalesce(first(count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124) ignore nulls#197, 0) AS B2_CNT#6, count(spark_catalog.default.store_sales.ss_list_price#31)#198 AS B2_CNTD#7, cast((first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125) ignore nulls#199 / 100.0) as decimal(11,6)) AS B3_LP#8, coalesce(first(count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126) ignore nulls#200, 0) AS B3_CNT#9, count(spark_catalog.default.store_sales.ss_list_price#31)#201 AS B3_CNTD#10, cast((first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127) ignore nulls#202 / 100.0) as decimal(11,6)) AS B4_LP#11, coalesce(first(count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128) ignore nulls#203, 0) AS B4_CNT#12, count(spark_catalog.default.store_sales.ss_list_price#31)#204 AS B4_CNTD#13, cast((first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129) ignore nulls#205 / 100.0) as decimal(11,6)) AS B5_LP#14, coalesce(first(count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130) ignore nulls#206, 0) AS B5_CNT#15, count(spark_catalog.default.store_sales.ss_list_price#31)#207 AS B5_CNTD#16, cast((first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131) ignore nulls#208 / 100.0) as decimal(11,6)) AS B6_LP#17, coalesce(first(count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132) ignore nulls#209, 0) AS B6_CNT#18, count(spark_catalog.default.store_sales.ss_list_price#31)#210 AS B6_CNTD#19] -(45) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] +(34) Project [codegen id : 3] +Output [1]: [named_struct(B1_LP, B1_LP#2, B1_CNT, B1_CNT#3, B1_CNTD, B1_CNTD#4, B2_LP, B2_LP#5, B2_CNT, B2_CNT#6, B2_CNTD, B2_CNTD#7, B3_LP, B3_LP#8, B3_CNT, B3_CNT#9, B3_CNTD, B3_CNTD#10, B4_LP, B4_LP#11, B4_CNT, B4_CNT#12, B4_CNTD, B4_CNTD#13, B5_LP, B5_LP#14, B5_CNT, B5_CNT#15, B5_CNTD, B5_CNTD#16, B6_LP, B6_LP#17, B6_CNT, B6_CNT#18, B6_CNTD, B6_CNTD#19) AS mergedValue#211] +Input [18]: [B1_LP#2, B1_CNT#3, B1_CNTD#4, B2_LP#5, B2_CNT#6, B2_CNTD#7, B3_LP#8, B3_CNT#9, B3_CNTD#10, B4_LP#11, B4_CNT#12, B4_CNTD#13, B5_LP#14, B5_CNT#15, B5_CNTD#16, B6_LP#17, B6_CNT#18, B6_CNTD#19] -(46) BroadcastNestedLoopJoin [codegen id : 18] -Join type: Inner -Join condition: None +Subquery:2 Hosting operator id = 2 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(47) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#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,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct +Subquery:3 Hosting operator id = 2 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] - -(49) Filter [codegen id : 12] -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(50) Project [codegen id : 12] -Output [1]: [ss_list_price#63] -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] - -(51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67] -Results [4]: [ss_list_price#63, sum#68, count#69, count#70] - -(52) Exchange -Input [4]: [ss_list_price#63, sum#68, count#69, count#70] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#63, sum#68, count#69, count#70] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67] -Results [4]: [ss_list_price#63, sum#68, count#69, count#70] - -(54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#63, sum#68, count#69, count#70] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67, count(ss_list_price#63)#71] -Results [4]: [sum#68, count#69, count#70, count#72] +Subquery:4 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(55) Exchange -Input [4]: [sum#68, count#69, count#70, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(56) HashAggregate [codegen id : 14] -Input [4]: [sum#68, count#69, count#70, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67, count(ss_list_price#63)#71] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#63))#66 / 100.0) as decimal(11,6)) AS B5_LP#73, count(ss_list_price#63)#67 AS B5_CNT#74, count(ss_list_price#63)#71 AS B5_CNTD#75] +Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(57) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] +Subquery:7 Hosting operator id = 8 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(58) BroadcastNestedLoopJoin [codegen id : 18] -Join type: Inner -Join condition: None +Subquery:8 Hosting operator id = 8 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(59) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct +Subquery:9 Hosting operator id = 8 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] - -(61) Filter [codegen id : 15] -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(62) Project [codegen id : 15] -Output [1]: [ss_list_price#78] -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] - -(63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82] -Results [4]: [ss_list_price#78, sum#83, count#84, count#85] - -(64) Exchange -Input [4]: [ss_list_price#78, sum#83, count#84, count#85] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#78, sum#83, count#84, count#85] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82] -Results [4]: [ss_list_price#78, sum#83, count#84, count#85] - -(66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#78, sum#83, count#84, count#85] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82, count(ss_list_price#78)#86] -Results [4]: [sum#83, count#84, count#85, count#87] +Subquery:10 Hosting operator id = 12 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(67) Exchange -Input [4]: [sum#83, count#84, count#85, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] +Subquery:11 Hosting operator id = 12 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(68) HashAggregate [codegen id : 17] -Input [4]: [sum#83, count#84, count#85, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82, count(ss_list_price#78)#86] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#78))#81 / 100.0) as decimal(11,6)) AS B6_LP#88, count(ss_list_price#78)#82 AS B6_CNT#89, count(ss_list_price#78)#86 AS B6_CNTD#90] +Subquery:12 Hosting operator id = 12 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(69) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] +Subquery:13 Hosting operator id = 16 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] + +Subquery:14 Hosting operator id = 16 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] + +Subquery:15 Hosting operator id = 16 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] + +Subquery:16 Hosting operator id = 20 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] + +Subquery:17 Hosting operator id = 20 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] + +Subquery:18 Hosting operator id = 20 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(70) BroadcastNestedLoopJoin [codegen id : 18] -Join type: Inner -Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/simplified.txt index 25f9d020f58f5..20ca85ba4dcb1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/simplified.txt @@ -1,111 +1,69 @@ -WholeStageCodegen (18) +WholeStageCodegen (6) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] - InputAdapter - Exchange #1 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #2 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + Project + Subquery #1 + WholeStageCodegen (3) + Project [B1_LP,B1_CNT,B1_CNTD,B2_LP,B2_CNT,B2_CNTD,B3_LP,B3_CNT,B3_CNTD,B4_LP,B4_CNT,B4_CNTD,B5_LP,B5_CNT,B5_CNTD,B6_LP,B6_CNT,B6_CNTD] + HashAggregate [first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count] [first(avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_1)) ignore nulls,first(count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_1)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_2)) ignore nulls,first(count(ss_list_price) FILTER (WHERE propagatedFilter_2)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)) ignore nulls,first(count(ss_list_price) FILTER (WHERE propagatedFilter_3)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)) ignore nulls,first(count(ss_list_price) FILTER (WHERE propagatedFilter_4)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)) ignore nulls,first(count(ss_list_price) FILTER (WHERE propagatedFilter_5)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)) ignore nulls,first(count(ss_list_price) FILTER (WHERE propagatedFilter_6)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),B1_LP,B1_CNT,B1_CNTD,B2_LP,B2_CNT,B2_CNTD,B3_LP,B3_CNT,B3_CNTD,B4_LP,B4_CNT,B4_CNTD,B5_LP,B5_CNT,B5_CNTD,B6_LP,B6_CNT,B6_CNTD,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count] + InputAdapter + Exchange #1 + WholeStageCodegen (2) + HashAggregate [avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_1),gid,count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_1),spark_catalog.default.store_sales.ss_list_price,propagatedFilter_1,avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_2),count(ss_list_price) FILTER (WHERE propagatedFilter_2),propagatedFilter_2,avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3),count(ss_list_price) FILTER (WHERE propagatedFilter_3),propagatedFilter_3,avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4),count(ss_list_price) FILTER (WHERE propagatedFilter_4),propagatedFilter_4,avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5),count(ss_list_price) FILTER (WHERE propagatedFilter_5),propagatedFilter_5,avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6),count(ss_list_price) FILTER (WHERE propagatedFilter_6),propagatedFilter_6] [first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count] + HashAggregate [spark_catalog.default.store_sales.ss_list_price,gid,max,max,max,max,max,max,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count] [max(propagatedFilter_1),max(propagatedFilter_2),max(propagatedFilter_3),max(propagatedFilter_4),max(propagatedFilter_5),max(propagatedFilter_6),avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)L),count(spark_catalog.default.store_sales.ss_list_price),avg(unscaledvalue(ss_list_price)L),count(ss_list_price),avg(unscaledvalue(ss_list_price)L),count(ss_list_price),avg(unscaledvalue(ss_list_price)L),count(ss_list_price),avg(unscaledvalue(ss_list_price)L),count(ss_list_price),avg(unscaledvalue(ss_list_price)L),count(ss_list_price),propagatedFilter_1,propagatedFilter_2,propagatedFilter_3,propagatedFilter_4,propagatedFilter_5,propagatedFilter_6,avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_1),count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_1),avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_2),count(ss_list_price) FILTER (WHERE propagatedFilter_2),avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3),count(ss_list_price) FILTER (WHERE propagatedFilter_3),avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4),count(ss_list_price) FILTER (WHERE propagatedFilter_4),avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5),count(ss_list_price) FILTER (WHERE propagatedFilter_5),avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6),count(ss_list_price) FILTER (WHERE propagatedFilter_6),max,max,max,max,max,max,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count] + InputAdapter + Exchange [spark_catalog.default.store_sales.ss_list_price,gid] #2 + WholeStageCodegen (1) + HashAggregate [spark_catalog.default.store_sales.ss_list_price,gid,propagatedFilter_1,propagatedFilter_2,propagatedFilter_3,propagatedFilter_4,propagatedFilter_5,propagatedFilter_6,unscaledvalue(spark_catalog.default.store_sales.ss_list_price),propagatedFilter_1,spark_catalog.default.store_sales.ss_list_price,unscaledvalue(ss_list_price),propagatedFilter_2,ss_list_price,propagatedFilter_3,propagatedFilter_4,propagatedFilter_5,propagatedFilter_6] [max,max,max,max,max,max,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count,max,max,max,max,max,max,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count] + Expand [ss_list_price,propagatedFilter_1,propagatedFilter_2,propagatedFilter_3,propagatedFilter_4,propagatedFilter_5,propagatedFilter_6] + Project [ss_list_price,ss_quantity,ss_coupon_amt,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation InputAdapter BroadcastExchange #3 - WholeStageCodegen (5) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] - InputAdapter - Exchange #4 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #5 - WholeStageCodegen (3) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + WholeStageCodegen (1) + Project + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] - InputAdapter - Exchange #7 - WholeStageCodegen (7) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #8 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastExchange #4 + WholeStageCodegen (2) + Project + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation InputAdapter - BroadcastExchange #9 - WholeStageCodegen (11) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] - InputAdapter - Exchange #10 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #11 - WholeStageCodegen (9) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastExchange #5 + WholeStageCodegen (3) + Project + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation InputAdapter - BroadcastExchange #12 - WholeStageCodegen (14) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] - InputAdapter - Exchange #13 - WholeStageCodegen (13) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #14 - WholeStageCodegen (12) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastExchange #6 + WholeStageCodegen (4) + Project + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation InputAdapter - BroadcastExchange #15 - WholeStageCodegen (17) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] - InputAdapter - Exchange #16 - WholeStageCodegen (16) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #17 - WholeStageCodegen (15) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastExchange #7 + WholeStageCodegen (5) + Project + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt index bfc684b1488d6..e9970ea0dd88d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt @@ -1,437 +1,221 @@ == Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (70) -:- * BroadcastNestedLoopJoin Inner BuildRight (58) -: :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (34) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (22) -: : : : :- * HashAggregate (10) -: : : : : +- Exchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- Exchange (6) -: : : : : +- * HashAggregate (5) -: : : : : +- * Project (4) -: : : : : +- * Filter (3) -: : : : : +- * ColumnarToRow (2) -: : : : : +- Scan parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (21) -: : : : +- * HashAggregate (20) -: : : : +- Exchange (19) -: : : : +- * HashAggregate (18) -: : : : +- * HashAggregate (17) -: : : : +- Exchange (16) -: : : : +- * HashAggregate (15) -: : : : +- * Project (14) -: : : : +- * Filter (13) -: : : : +- * ColumnarToRow (12) -: : : : +- Scan parquet spark_catalog.default.store_sales (11) -: : : +- BroadcastExchange (33) -: : : +- * HashAggregate (32) -: : : +- Exchange (31) -: : : +- * HashAggregate (30) -: : : +- * HashAggregate (29) -: : : +- Exchange (28) -: : : +- * HashAggregate (27) -: : : +- * Project (26) -: : : +- * Filter (25) -: : : +- * ColumnarToRow (24) -: : : +- Scan parquet spark_catalog.default.store_sales (23) -: : +- BroadcastExchange (45) -: : +- * HashAggregate (44) -: : +- Exchange (43) -: : +- * HashAggregate (42) -: : +- * HashAggregate (41) -: : +- Exchange (40) -: : +- * HashAggregate (39) -: : +- * Project (38) -: : +- * Filter (37) -: : +- * ColumnarToRow (36) -: : +- Scan parquet spark_catalog.default.store_sales (35) -: +- BroadcastExchange (57) -: +- * HashAggregate (56) -: +- Exchange (55) -: +- * HashAggregate (54) -: +- * HashAggregate (53) -: +- Exchange (52) -: +- * HashAggregate (51) -: +- * Project (50) -: +- * Filter (49) -: +- * ColumnarToRow (48) -: +- Scan parquet spark_catalog.default.store_sales (47) -+- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet spark_catalog.default.store_sales (59) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct +* BroadcastNestedLoopJoin Inner BuildRight (22) +:- * BroadcastNestedLoopJoin Inner BuildRight (18) +: :- * BroadcastNestedLoopJoin Inner BuildRight (14) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (10) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (6) +: : : : :- * Project (2) +: : : : : +- * Scan OneRowRelation (1) +: : : : +- BroadcastExchange (5) +: : : : +- * Project (4) +: : : : +- * Scan OneRowRelation (3) +: : : +- BroadcastExchange (9) +: : : +- * Project (8) +: : : +- * Scan OneRowRelation (7) +: : +- BroadcastExchange (13) +: : +- * Project (12) +: : +- * Scan OneRowRelation (11) +: +- BroadcastExchange (17) +: +- * Project (16) +: +- * Scan OneRowRelation (15) ++- BroadcastExchange (21) + +- * Project (20) + +- * Scan OneRowRelation (19) + + +(1) Scan OneRowRelation [codegen id : 6] +Output: [] + +(2) Project [codegen id : 6] +Output [3]: [Subquery scalar-subquery#1, [id=#1].B1_LP AS B1_LP#2, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B1_CNT AS B1_CNT#3, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B1_CNTD AS B1_CNTD#4] +Input: [] + +(3) Scan OneRowRelation [codegen id : 1] +Output: [] -(2) ColumnarToRow [codegen id : 1] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +(4) Project [codegen id : 1] +Output [3]: [ReusedSubquery Subquery scalar-subquery#1, [id=#1].B2_LP AS B2_LP#5, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B2_CNT AS B2_CNT#6, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B2_CNTD AS B2_CNTD#7] +Input: [] -(3) Filter [codegen id : 1] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) +(5) BroadcastExchange +Input [3]: [B2_LP#5, B2_CNT#6, B2_CNTD#7] +Arguments: IdentityBroadcastMode, [plan_id=2] -(4) Project [codegen id : 1] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] - -(5) HashAggregate [codegen id : 1] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7] -Results [4]: [ss_list_price#3, sum#8, count#9, count#10] - -(6) Exchange -Input [4]: [ss_list_price#3, sum#8, count#9, count#10] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(7) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#3, sum#8, count#9, count#10] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7] -Results [4]: [ss_list_price#3, sum#8, count#9, count#10] - -(8) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#3, sum#8, count#9, count#10] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7, count(ss_list_price#3)#11] -Results [4]: [sum#8, count#9, count#10, count#12] +(6) BroadcastNestedLoopJoin [codegen id : 6] +Join type: Inner +Join condition: None -(9) Exchange -Input [4]: [sum#8, count#9, count#10, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +(7) Scan OneRowRelation [codegen id : 2] +Output: [] -(10) HashAggregate [codegen id : 18] -Input [4]: [sum#8, count#9, count#10, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7, count(ss_list_price#3)#11] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#13, count(ss_list_price#3)#7 AS B1_CNT#14, count(ss_list_price#3)#11 AS B1_CNTD#15] +(8) Project [codegen id : 2] +Output [3]: [ReusedSubquery Subquery scalar-subquery#1, [id=#1].B3_LP AS B3_LP#8, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B3_CNT AS B3_CNT#9, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B3_CNTD AS B3_CNTD#10] +Input: [] -(11) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct +(9) BroadcastExchange +Input [3]: [B3_LP#8, B3_CNT#9, B3_CNTD#10] +Arguments: IdentityBroadcastMode, [plan_id=3] -(12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] - -(13) Filter [codegen id : 3] -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) Project [codegen id : 3] -Output [1]: [ss_list_price#18] -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] - -(15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22] -Results [4]: [ss_list_price#18, sum#23, count#24, count#25] - -(16) Exchange -Input [4]: [ss_list_price#18, sum#23, count#24, count#25] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#18, sum#23, count#24, count#25] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22] -Results [4]: [ss_list_price#18, sum#23, count#24, count#25] - -(18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#18, sum#23, count#24, count#25] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22, count(ss_list_price#18)#26] -Results [4]: [sum#23, count#24, count#25, count#27] +(10) BroadcastNestedLoopJoin [codegen id : 6] +Join type: Inner +Join condition: None -(19) Exchange -Input [4]: [sum#23, count#24, count#25, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(11) Scan OneRowRelation [codegen id : 3] +Output: [] -(20) HashAggregate [codegen id : 5] -Input [4]: [sum#23, count#24, count#25, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22, count(ss_list_price#18)#26] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#18))#21 / 100.0) as decimal(11,6)) AS B2_LP#28, count(ss_list_price#18)#22 AS B2_CNT#29, count(ss_list_price#18)#26 AS B2_CNTD#30] +(12) Project [codegen id : 3] +Output [3]: [ReusedSubquery Subquery scalar-subquery#1, [id=#1].B4_LP AS B4_LP#11, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B4_CNT AS B4_CNT#12, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B4_CNTD AS B4_CNTD#13] +Input: [] -(21) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] +(13) BroadcastExchange +Input [3]: [B4_LP#11, B4_CNT#12, B4_CNTD#13] +Arguments: IdentityBroadcastMode, [plan_id=4] -(22) BroadcastNestedLoopJoin [codegen id : 18] +(14) BroadcastNestedLoopJoin [codegen id : 6] Join type: Inner Join condition: None -(23) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct +(15) Scan OneRowRelation [codegen id : 4] +Output: [] -(24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] - -(25) Filter [codegen id : 6] -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(26) Project [codegen id : 6] -Output [1]: [ss_list_price#33] -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] - -(27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37] -Results [4]: [ss_list_price#33, sum#38, count#39, count#40] - -(28) Exchange -Input [4]: [ss_list_price#33, sum#38, count#39, count#40] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#33, sum#38, count#39, count#40] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37] -Results [4]: [ss_list_price#33, sum#38, count#39, count#40] - -(30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#33, sum#38, count#39, count#40] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37, count(ss_list_price#33)#41] -Results [4]: [sum#38, count#39, count#40, count#42] +(16) Project [codegen id : 4] +Output [3]: [ReusedSubquery Subquery scalar-subquery#1, [id=#1].B5_LP AS B5_LP#14, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B5_CNT AS B5_CNT#15, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B5_CNTD AS B5_CNTD#16] +Input: [] -(31) Exchange -Input [4]: [sum#38, count#39, count#40, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(17) BroadcastExchange +Input [3]: [B5_LP#14, B5_CNT#15, B5_CNTD#16] +Arguments: IdentityBroadcastMode, [plan_id=5] -(32) HashAggregate [codegen id : 8] -Input [4]: [sum#38, count#39, count#40, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37, count(ss_list_price#33)#41] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#33))#36 / 100.0) as decimal(11,6)) AS B3_LP#43, count(ss_list_price#33)#37 AS B3_CNT#44, count(ss_list_price#33)#41 AS B3_CNTD#45] +(18) BroadcastNestedLoopJoin [codegen id : 6] +Join type: Inner +Join condition: None + +(19) Scan OneRowRelation [codegen id : 5] +Output: [] -(33) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] +(20) Project [codegen id : 5] +Output [3]: [ReusedSubquery Subquery scalar-subquery#1, [id=#1].B6_LP AS B6_LP#17, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B6_CNT AS B6_CNT#18, ReusedSubquery Subquery scalar-subquery#1, [id=#1].B6_CNTD AS B6_CNTD#19] +Input: [] -(34) BroadcastNestedLoopJoin [codegen id : 18] +(21) BroadcastExchange +Input [3]: [B6_LP#17, B6_CNT#18, B6_CNTD#19] +Arguments: IdentityBroadcastMode, [plan_id=6] + +(22) BroadcastNestedLoopJoin [codegen id : 6] Join type: Inner Join condition: None -(35) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] +===== Subqueries ===== + +Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery scalar-subquery#1, [id=#1] +* Project (34) ++- * HashAggregate (33) + +- Exchange (32) + +- * HashAggregate (31) + +- * HashAggregate (30) + +- Exchange (29) + +- * HashAggregate (28) + +- * Expand (27) + +- * Project (26) + +- * Filter (25) + +- * ColumnarToRow (24) + +- Scan parquet spark_catalog.default.store_sales (23) + + +(23) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_quantity#20, ss_wholesale_cost#21, ss_list_price#22, ss_coupon_amt#23, ss_sold_date_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] +PushedFilters: [IsNotNull(ss_quantity), Or(Or(Or(Or(Or(And(And(GreaterThanOrEqual(ss_quantity,6),LessThanOrEqual(ss_quantity,10)),Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))),And(And(GreaterThanOrEqual(ss_quantity,0),LessThanOrEqual(ss_quantity,5)),Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00))))),And(And(GreaterThanOrEqual(ss_quantity,11),LessThanOrEqual(ss_quantity,15)),Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00))))),And(And(GreaterThanOrEqual(ss_quantity,16),LessThanOrEqual(ss_quantity,20)),Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00))))),And(And(GreaterThanOrEqual(ss_quantity,21),LessThanOrEqual(ss_quantity,25)),Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00))))),And(And(GreaterThanOrEqual(ss_quantity,26),LessThanOrEqual(ss_quantity,30)),Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))))] ReadSchema: struct -(36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] - -(37) Filter [codegen id : 9] -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(38) Project [codegen id : 9] -Output [1]: [ss_list_price#48] -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] - -(39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52] -Results [4]: [ss_list_price#48, sum#53, count#54, count#55] - -(40) Exchange -Input [4]: [ss_list_price#48, sum#53, count#54, count#55] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#48, sum#53, count#54, count#55] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52] -Results [4]: [ss_list_price#48, sum#53, count#54, count#55] - -(42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#48, sum#53, count#54, count#55] +(24) ColumnarToRow [codegen id : 1] +Input [5]: [ss_quantity#20, ss_wholesale_cost#21, ss_list_price#22, ss_coupon_amt#23, ss_sold_date_sk#24] + +(25) Filter [codegen id : 1] +Input [5]: [ss_quantity#20, ss_wholesale_cost#21, ss_list_price#22, ss_coupon_amt#23, ss_sold_date_sk#24] +Condition : (isnotnull(ss_quantity#20) AND ((((((((ss_quantity#20 >= 6) AND (ss_quantity#20 <= 10)) AND ((((ss_list_price#22 >= 90.00) AND (ss_list_price#22 <= 100.00)) OR ((ss_coupon_amt#23 >= 2323.00) AND (ss_coupon_amt#23 <= 3323.00))) OR ((ss_wholesale_cost#21 >= 31.00) AND (ss_wholesale_cost#21 <= 51.00)))) OR (((ss_quantity#20 >= 0) AND (ss_quantity#20 <= 5)) AND ((((ss_list_price#22 >= 8.00) AND (ss_list_price#22 <= 18.00)) OR ((ss_coupon_amt#23 >= 459.00) AND (ss_coupon_amt#23 <= 1459.00))) OR ((ss_wholesale_cost#21 >= 57.00) AND (ss_wholesale_cost#21 <= 77.00))))) OR (((ss_quantity#20 >= 11) AND (ss_quantity#20 <= 15)) AND ((((ss_list_price#22 >= 142.00) AND (ss_list_price#22 <= 152.00)) OR ((ss_coupon_amt#23 >= 12214.00) AND (ss_coupon_amt#23 <= 13214.00))) OR ((ss_wholesale_cost#21 >= 79.00) AND (ss_wholesale_cost#21 <= 99.00))))) OR (((ss_quantity#20 >= 16) AND (ss_quantity#20 <= 20)) AND ((((ss_list_price#22 >= 135.00) AND (ss_list_price#22 <= 145.00)) OR ((ss_coupon_amt#23 >= 6071.00) AND (ss_coupon_amt#23 <= 7071.00))) OR ((ss_wholesale_cost#21 >= 38.00) AND (ss_wholesale_cost#21 <= 58.00))))) OR (((ss_quantity#20 >= 21) AND (ss_quantity#20 <= 25)) AND ((((ss_list_price#22 >= 122.00) AND (ss_list_price#22 <= 132.00)) OR ((ss_coupon_amt#23 >= 836.00) AND (ss_coupon_amt#23 <= 1836.00))) OR ((ss_wholesale_cost#21 >= 17.00) AND (ss_wholesale_cost#21 <= 37.00))))) OR (((ss_quantity#20 >= 26) AND (ss_quantity#20 <= 30)) AND ((((ss_list_price#22 >= 154.00) AND (ss_list_price#22 <= 164.00)) OR ((ss_coupon_amt#23 >= 7326.00) AND (ss_coupon_amt#23 <= 8326.00))) OR ((ss_wholesale_cost#21 >= 7.00) AND (ss_wholesale_cost#21 <= 27.00)))))) + +(26) Project [codegen id : 1] +Output [7]: [ss_list_price#22, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 6) AND (ss_quantity#20 <= 10)) AND ((((ss_list_price#22 >= 90.00) AND (ss_list_price#22 <= 100.00)) OR ((ss_coupon_amt#23 >= 2323.00) AND (ss_coupon_amt#23 <= 3323.00))) OR ((ss_wholesale_cost#21 >= 31.00) AND (ss_wholesale_cost#21 <= 51.00))))) AS propagatedFilter_1#25, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 0) AND (ss_quantity#20 <= 5)) AND ((((ss_list_price#22 >= 8.00) AND (ss_list_price#22 <= 18.00)) OR ((ss_coupon_amt#23 >= 459.00) AND (ss_coupon_amt#23 <= 1459.00))) OR ((ss_wholesale_cost#21 >= 57.00) AND (ss_wholesale_cost#21 <= 77.00))))) AS propagatedFilter_2#26, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 11) AND (ss_quantity#20 <= 15)) AND ((((ss_list_price#22 >= 142.00) AND (ss_list_price#22 <= 152.00)) OR ((ss_coupon_amt#23 >= 12214.00) AND (ss_coupon_amt#23 <= 13214.00))) OR ((ss_wholesale_cost#21 >= 79.00) AND (ss_wholesale_cost#21 <= 99.00))))) AS propagatedFilter_3#27, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 16) AND (ss_quantity#20 <= 20)) AND ((((ss_list_price#22 >= 135.00) AND (ss_list_price#22 <= 145.00)) OR ((ss_coupon_amt#23 >= 6071.00) AND (ss_coupon_amt#23 <= 7071.00))) OR ((ss_wholesale_cost#21 >= 38.00) AND (ss_wholesale_cost#21 <= 58.00))))) AS propagatedFilter_4#28, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 21) AND (ss_quantity#20 <= 25)) AND ((((ss_list_price#22 >= 122.00) AND (ss_list_price#22 <= 132.00)) OR ((ss_coupon_amt#23 >= 836.00) AND (ss_coupon_amt#23 <= 1836.00))) OR ((ss_wholesale_cost#21 >= 17.00) AND (ss_wholesale_cost#21 <= 37.00))))) AS propagatedFilter_5#29, (isnotnull(ss_quantity#20) AND (((ss_quantity#20 >= 26) AND (ss_quantity#20 <= 30)) AND ((((ss_list_price#22 >= 154.00) AND (ss_list_price#22 <= 164.00)) OR ((ss_coupon_amt#23 >= 7326.00) AND (ss_coupon_amt#23 <= 8326.00))) OR ((ss_wholesale_cost#21 >= 7.00) AND (ss_wholesale_cost#21 <= 27.00))))) AS propagatedFilter_6#30] +Input [5]: [ss_quantity#20, ss_wholesale_cost#21, ss_list_price#22, ss_coupon_amt#23, ss_sold_date_sk#24] + +(27) Expand [codegen id : 1] +Input [7]: [ss_list_price#22, propagatedFilter_1#25, propagatedFilter_2#26, propagatedFilter_3#27, propagatedFilter_4#28, propagatedFilter_5#29, propagatedFilter_6#30] +Arguments: [[null, 0, null, null, null, null, null, null, UnscaledValue(ss_list_price#22), ss_list_price#22, UnscaledValue(ss_list_price#22), ss_list_price#22, propagatedFilter_2#26, propagatedFilter_1#25, propagatedFilter_3#27, propagatedFilter_4#28, propagatedFilter_5#29, propagatedFilter_6#30], [ss_list_price#22, 1, propagatedFilter_2#26, propagatedFilter_1#25, propagatedFilter_3#27, propagatedFilter_4#28, propagatedFilter_5#29, propagatedFilter_6#30, null, null, null, null, null, null, null, null, null, null]], [spark_catalog.default.store_sales.ss_list_price#31, gid#32, propagatedFilter_2#33, propagatedFilter_1#34, propagatedFilter_3#35, propagatedFilter_4#36, propagatedFilter_5#37, propagatedFilter_6#38, unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39, spark_catalog.default.store_sales.ss_list_price#40, unscaledvalue(ss_list_price)#41, ss_list_price#42, propagatedFilter_2#43, propagatedFilter_1#44, propagatedFilter_3#45, propagatedFilter_4#46, propagatedFilter_5#47, propagatedFilter_6#48] + +(28) HashAggregate [codegen id : 1] +Input [18]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, propagatedFilter_2#33, propagatedFilter_1#34, propagatedFilter_3#35, propagatedFilter_4#36, propagatedFilter_5#37, propagatedFilter_6#38, unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39, spark_catalog.default.store_sales.ss_list_price#40, unscaledvalue(ss_list_price)#41, ss_list_price#42, propagatedFilter_2#43, propagatedFilter_1#44, propagatedFilter_3#45, propagatedFilter_4#46, propagatedFilter_5#47, propagatedFilter_6#48] +Keys [2]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32] +Functions [18]: [partial_max(propagatedFilter_2#33), partial_max(propagatedFilter_1#34), partial_max(propagatedFilter_3#35), partial_max(propagatedFilter_4#36), partial_max(propagatedFilter_5#37), partial_max(propagatedFilter_6#38), partial_avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39) FILTER (WHERE propagatedFilter_2#43), partial_count(spark_catalog.default.store_sales.ss_list_price#40) FILTER (WHERE propagatedFilter_2#43), partial_avg(unscaledvalue(ss_list_price)#41) FILTER (WHERE propagatedFilter_1#44), partial_count(ss_list_price#42) FILTER (WHERE propagatedFilter_1#44), partial_avg(unscaledvalue(ss_list_price)#41) FILTER (WHERE propagatedFilter_3#45), partial_count(ss_list_price#42) FILTER (WHERE propagatedFilter_3#45), partial_avg(unscaledvalue(ss_list_price)#41) FILTER (WHERE propagatedFilter_4#46), partial_count(ss_list_price#42) FILTER (WHERE propagatedFilter_4#46), partial_avg(unscaledvalue(ss_list_price)#41) FILTER (WHERE propagatedFilter_5#47), partial_count(ss_list_price#42) FILTER (WHERE propagatedFilter_5#47), partial_avg(unscaledvalue(ss_list_price)#41) FILTER (WHERE propagatedFilter_6#48), partial_count(ss_list_price#42) FILTER (WHERE propagatedFilter_6#48)] +Aggregate Attributes [24]: [max#49, max#50, max#51, max#52, max#53, max#54, sum#55, count#56, count#57, sum#58, count#59, count#60, sum#61, count#62, count#63, sum#64, count#65, count#66, sum#67, count#68, count#69, sum#70, count#71, count#72] +Results [26]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, max#73, max#74, max#75, max#76, max#77, max#78, sum#79, count#80, count#81, sum#82, count#83, count#84, sum#85, count#86, count#87, sum#88, count#89, count#90, sum#91, count#92, count#93, sum#94, count#95, count#96] + +(29) Exchange +Input [26]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, max#73, max#74, max#75, max#76, max#77, max#78, sum#79, count#80, count#81, sum#82, count#83, count#84, sum#85, count#86, count#87, sum#88, count#89, count#90, sum#91, count#92, count#93, sum#94, count#95, count#96] +Arguments: hashpartitioning(spark_catalog.default.store_sales.ss_list_price#31, gid#32, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(30) HashAggregate [codegen id : 2] +Input [26]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, max#73, max#74, max#75, max#76, max#77, max#78, sum#79, count#80, count#81, sum#82, count#83, count#84, sum#85, count#86, count#87, sum#88, count#89, count#90, sum#91, count#92, count#93, sum#94, count#95, count#96] +Keys [2]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32] +Functions [18]: [max(propagatedFilter_2#33), max(propagatedFilter_1#34), max(propagatedFilter_3#35), max(propagatedFilter_4#36), max(propagatedFilter_5#37), max(propagatedFilter_6#38), avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39), count(spark_catalog.default.store_sales.ss_list_price#40), avg(unscaledvalue(ss_list_price)#41), count(ss_list_price#42), avg(unscaledvalue(ss_list_price)#41), count(ss_list_price#42), avg(unscaledvalue(ss_list_price)#41), count(ss_list_price#42), avg(unscaledvalue(ss_list_price)#41), count(ss_list_price#42), avg(unscaledvalue(ss_list_price)#41), count(ss_list_price#42)] +Aggregate Attributes [18]: [max(propagatedFilter_2#33)#97, max(propagatedFilter_1#34)#98, max(propagatedFilter_3#35)#99, max(propagatedFilter_4#36)#100, max(propagatedFilter_5#37)#101, max(propagatedFilter_6#38)#102, avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39)#103, count(spark_catalog.default.store_sales.ss_list_price#40)#104, avg(unscaledvalue(ss_list_price)#41)#105, count(ss_list_price#42)#106, avg(unscaledvalue(ss_list_price)#41)#107, count(ss_list_price#42)#108, avg(unscaledvalue(ss_list_price)#41)#109, count(ss_list_price#42)#110, avg(unscaledvalue(ss_list_price)#41)#111, count(ss_list_price#42)#112, avg(unscaledvalue(ss_list_price)#41)#113, count(ss_list_price#42)#114] +Results [20]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, max(propagatedFilter_2#33)#97 AS propagatedFilter_2#115, max(propagatedFilter_1#34)#98 AS propagatedFilter_1#116, max(propagatedFilter_3#35)#99 AS propagatedFilter_3#117, max(propagatedFilter_4#36)#100 AS propagatedFilter_4#118, max(propagatedFilter_5#37)#101 AS propagatedFilter_5#119, max(propagatedFilter_6#38)#102 AS propagatedFilter_6#120, avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)#39)#103 AS avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121, count(spark_catalog.default.store_sales.ss_list_price#40)#104 AS count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122, avg(unscaledvalue(ss_list_price)#41)#105 AS avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123, count(ss_list_price#42)#106 AS count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124, avg(unscaledvalue(ss_list_price)#41)#107 AS avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125, count(ss_list_price#42)#108 AS count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126, avg(unscaledvalue(ss_list_price)#41)#109 AS avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127, count(ss_list_price#42)#110 AS count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128, avg(unscaledvalue(ss_list_price)#41)#111 AS avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129, count(ss_list_price#42)#112 AS count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130, avg(unscaledvalue(ss_list_price)#41)#113 AS avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131, count(ss_list_price#42)#114 AS count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132] + +(31) HashAggregate [codegen id : 2] +Input [20]: [spark_catalog.default.store_sales.ss_list_price#31, gid#32, propagatedFilter_2#115, propagatedFilter_1#116, propagatedFilter_3#117, propagatedFilter_4#118, propagatedFilter_5#119, propagatedFilter_6#120, avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121, count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122, avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123, count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124, avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125, count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126, avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127, count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128, avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129, count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130, avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131, count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52, count(ss_list_price#48)#56] -Results [4]: [sum#53, count#54, count#55, count#57] +Functions [18]: [partial_first(avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_2#115)), partial_first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_1#116)), partial_first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_3#117)), partial_first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_4#118)), partial_first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_5#119)), partial_first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131, true) FILTER (WHERE (gid#32 = 0)), partial_first(count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132, true) FILTER (WHERE (gid#32 = 0)), partial_count(spark_catalog.default.store_sales.ss_list_price#31) FILTER (WHERE ((gid#32 = 1) AND propagatedFilter_6#120))] +Aggregate Attributes [30]: [first#133, valueSet#134, first#135, valueSet#136, count#137, first#138, valueSet#139, first#140, valueSet#141, count#142, first#143, valueSet#144, first#145, valueSet#146, count#147, first#148, valueSet#149, first#150, valueSet#151, count#152, first#153, valueSet#154, first#155, valueSet#156, count#157, first#158, valueSet#159, first#160, valueSet#161, count#162] +Results [30]: [first#163, valueSet#164, first#165, valueSet#166, count#167, first#168, valueSet#169, first#170, valueSet#171, count#172, first#173, valueSet#174, first#175, valueSet#176, count#177, first#178, valueSet#179, first#180, valueSet#181, count#182, first#183, valueSet#184, first#185, valueSet#186, count#187, first#188, valueSet#189, first#190, valueSet#191, count#192] -(43) Exchange -Input [4]: [sum#53, count#54, count#55, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] +(32) Exchange +Input [30]: [first#163, valueSet#164, first#165, valueSet#166, count#167, first#168, valueSet#169, first#170, valueSet#171, count#172, first#173, valueSet#174, first#175, valueSet#176, count#177, first#178, valueSet#179, first#180, valueSet#181, count#182, first#183, valueSet#184, first#185, valueSet#186, count#187, first#188, valueSet#189, first#190, valueSet#191, count#192] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(44) HashAggregate [codegen id : 11] -Input [4]: [sum#53, count#54, count#55, count#57] +(33) HashAggregate [codegen id : 3] +Input [30]: [first#163, valueSet#164, first#165, valueSet#166, count#167, first#168, valueSet#169, first#170, valueSet#171, count#172, first#173, valueSet#174, first#175, valueSet#176, count#177, first#178, valueSet#179, first#180, valueSet#181, count#182, first#183, valueSet#184, first#185, valueSet#186, count#187, first#188, valueSet#189, first#190, valueSet#191, count#192] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52, count(ss_list_price#48)#56] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#48))#51 / 100.0) as decimal(11,6)) AS B4_LP#58, count(ss_list_price#48)#52 AS B4_CNT#59, count(ss_list_price#48)#56 AS B4_CNTD#60] +Functions [18]: [first(avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121, true), first(count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122, true), count(spark_catalog.default.store_sales.ss_list_price#31), first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123, true), first(count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124, true), count(spark_catalog.default.store_sales.ss_list_price#31), first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125, true), first(count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126, true), count(spark_catalog.default.store_sales.ss_list_price#31), first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127, true), first(count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128, true), count(spark_catalog.default.store_sales.ss_list_price#31), first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129, true), first(count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130, true), count(spark_catalog.default.store_sales.ss_list_price#31), first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131, true), first(count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132, true), count(spark_catalog.default.store_sales.ss_list_price#31)] +Aggregate Attributes [18]: [first(avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121) ignore nulls#193, first(count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122) ignore nulls#194, count(spark_catalog.default.store_sales.ss_list_price#31)#195, first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123) ignore nulls#196, first(count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124) ignore nulls#197, count(spark_catalog.default.store_sales.ss_list_price#31)#198, first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125) ignore nulls#199, first(count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126) ignore nulls#200, count(spark_catalog.default.store_sales.ss_list_price#31)#201, first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127) ignore nulls#202, first(count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128) ignore nulls#203, count(spark_catalog.default.store_sales.ss_list_price#31)#204, first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129) ignore nulls#205, first(count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130) ignore nulls#206, count(spark_catalog.default.store_sales.ss_list_price#31)#207, first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131) ignore nulls#208, first(count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132) ignore nulls#209, count(spark_catalog.default.store_sales.ss_list_price#31)#210] +Results [18]: [cast((first(avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_2)#121) ignore nulls#193 / 100.0) as decimal(11,6)) AS B1_LP#2, coalesce(first(count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_2)#122) ignore nulls#194, 0) AS B1_CNT#3, count(spark_catalog.default.store_sales.ss_list_price#31)#195 AS B1_CNTD#4, cast((first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_1)#123) ignore nulls#196 / 100.0) as decimal(11,6)) AS B2_LP#5, coalesce(first(count(ss_list_price) FILTER (WHERE propagatedFilter_1)#124) ignore nulls#197, 0) AS B2_CNT#6, count(spark_catalog.default.store_sales.ss_list_price#31)#198 AS B2_CNTD#7, cast((first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)#125) ignore nulls#199 / 100.0) as decimal(11,6)) AS B3_LP#8, coalesce(first(count(ss_list_price) FILTER (WHERE propagatedFilter_3)#126) ignore nulls#200, 0) AS B3_CNT#9, count(spark_catalog.default.store_sales.ss_list_price#31)#201 AS B3_CNTD#10, cast((first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)#127) ignore nulls#202 / 100.0) as decimal(11,6)) AS B4_LP#11, coalesce(first(count(ss_list_price) FILTER (WHERE propagatedFilter_4)#128) ignore nulls#203, 0) AS B4_CNT#12, count(spark_catalog.default.store_sales.ss_list_price#31)#204 AS B4_CNTD#13, cast((first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)#129) ignore nulls#205 / 100.0) as decimal(11,6)) AS B5_LP#14, coalesce(first(count(ss_list_price) FILTER (WHERE propagatedFilter_5)#130) ignore nulls#206, 0) AS B5_CNT#15, count(spark_catalog.default.store_sales.ss_list_price#31)#207 AS B5_CNTD#16, cast((first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)#131) ignore nulls#208 / 100.0) as decimal(11,6)) AS B6_LP#17, coalesce(first(count(ss_list_price) FILTER (WHERE propagatedFilter_6)#132) ignore nulls#209, 0) AS B6_CNT#18, count(spark_catalog.default.store_sales.ss_list_price#31)#210 AS B6_CNTD#19] -(45) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] +(34) Project [codegen id : 3] +Output [1]: [named_struct(B1_LP, B1_LP#2, B1_CNT, B1_CNT#3, B1_CNTD, B1_CNTD#4, B2_LP, B2_LP#5, B2_CNT, B2_CNT#6, B2_CNTD, B2_CNTD#7, B3_LP, B3_LP#8, B3_CNT, B3_CNT#9, B3_CNTD, B3_CNTD#10, B4_LP, B4_LP#11, B4_CNT, B4_CNT#12, B4_CNTD, B4_CNTD#13, B5_LP, B5_LP#14, B5_CNT, B5_CNT#15, B5_CNTD, B5_CNTD#16, B6_LP, B6_LP#17, B6_CNT, B6_CNT#18, B6_CNTD, B6_CNTD#19) AS mergedValue#211] +Input [18]: [B1_LP#2, B1_CNT#3, B1_CNTD#4, B2_LP#5, B2_CNT#6, B2_CNTD#7, B3_LP#8, B3_CNT#9, B3_CNTD#10, B4_LP#11, B4_CNT#12, B4_CNTD#13, B5_LP#14, B5_CNT#15, B5_CNTD#16, B6_LP#17, B6_CNT#18, B6_CNTD#19] -(46) BroadcastNestedLoopJoin [codegen id : 18] -Join type: Inner -Join condition: None +Subquery:2 Hosting operator id = 2 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(47) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#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,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct +Subquery:3 Hosting operator id = 2 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] - -(49) Filter [codegen id : 12] -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(50) Project [codegen id : 12] -Output [1]: [ss_list_price#63] -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] - -(51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67] -Results [4]: [ss_list_price#63, sum#68, count#69, count#70] - -(52) Exchange -Input [4]: [ss_list_price#63, sum#68, count#69, count#70] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#63, sum#68, count#69, count#70] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67] -Results [4]: [ss_list_price#63, sum#68, count#69, count#70] - -(54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#63, sum#68, count#69, count#70] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67, count(ss_list_price#63)#71] -Results [4]: [sum#68, count#69, count#70, count#72] +Subquery:4 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(55) Exchange -Input [4]: [sum#68, count#69, count#70, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(56) HashAggregate [codegen id : 14] -Input [4]: [sum#68, count#69, count#70, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67, count(ss_list_price#63)#71] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#63))#66 / 100.0) as decimal(11,6)) AS B5_LP#73, count(ss_list_price#63)#67 AS B5_CNT#74, count(ss_list_price#63)#71 AS B5_CNTD#75] +Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(57) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] +Subquery:7 Hosting operator id = 8 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(58) BroadcastNestedLoopJoin [codegen id : 18] -Join type: Inner -Join condition: None +Subquery:8 Hosting operator id = 8 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(59) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct +Subquery:9 Hosting operator id = 8 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] - -(61) Filter [codegen id : 15] -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(62) Project [codegen id : 15] -Output [1]: [ss_list_price#78] -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] - -(63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82] -Results [4]: [ss_list_price#78, sum#83, count#84, count#85] - -(64) Exchange -Input [4]: [ss_list_price#78, sum#83, count#84, count#85] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#78, sum#83, count#84, count#85] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82] -Results [4]: [ss_list_price#78, sum#83, count#84, count#85] - -(66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#78, sum#83, count#84, count#85] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82, count(ss_list_price#78)#86] -Results [4]: [sum#83, count#84, count#85, count#87] +Subquery:10 Hosting operator id = 12 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(67) Exchange -Input [4]: [sum#83, count#84, count#85, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] +Subquery:11 Hosting operator id = 12 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(68) HashAggregate [codegen id : 17] -Input [4]: [sum#83, count#84, count#85, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82, count(ss_list_price#78)#86] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#78))#81 / 100.0) as decimal(11,6)) AS B6_LP#88, count(ss_list_price#78)#82 AS B6_CNT#89, count(ss_list_price#78)#86 AS B6_CNTD#90] +Subquery:12 Hosting operator id = 12 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(69) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] +Subquery:13 Hosting operator id = 16 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] + +Subquery:14 Hosting operator id = 16 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] + +Subquery:15 Hosting operator id = 16 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] + +Subquery:16 Hosting operator id = 20 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] + +Subquery:17 Hosting operator id = 20 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] + +Subquery:18 Hosting operator id = 20 Hosting Expression = ReusedSubquery Subquery scalar-subquery#1, [id=#1] -(70) BroadcastNestedLoopJoin [codegen id : 18] -Join type: Inner -Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index 25f9d020f58f5..20ca85ba4dcb1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -1,111 +1,69 @@ -WholeStageCodegen (18) +WholeStageCodegen (6) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] - InputAdapter - Exchange #1 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #2 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + Project + Subquery #1 + WholeStageCodegen (3) + Project [B1_LP,B1_CNT,B1_CNTD,B2_LP,B2_CNT,B2_CNTD,B3_LP,B3_CNT,B3_CNTD,B4_LP,B4_CNT,B4_CNTD,B5_LP,B5_CNT,B5_CNTD,B6_LP,B6_CNT,B6_CNTD] + HashAggregate [first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count] [first(avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_1)) ignore nulls,first(count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_1)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_2)) ignore nulls,first(count(ss_list_price) FILTER (WHERE propagatedFilter_2)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3)) ignore nulls,first(count(ss_list_price) FILTER (WHERE propagatedFilter_3)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4)) ignore nulls,first(count(ss_list_price) FILTER (WHERE propagatedFilter_4)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5)) ignore nulls,first(count(ss_list_price) FILTER (WHERE propagatedFilter_5)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),first(avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6)) ignore nulls,first(count(ss_list_price) FILTER (WHERE propagatedFilter_6)L) ignore nulls,count(spark_catalog.default.store_sales.ss_list_price),B1_LP,B1_CNT,B1_CNTD,B2_LP,B2_CNT,B2_CNTD,B3_LP,B3_CNT,B3_CNTD,B4_LP,B4_CNT,B4_CNTD,B5_LP,B5_CNT,B5_CNTD,B6_LP,B6_CNT,B6_CNTD,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count] + InputAdapter + Exchange #1 + WholeStageCodegen (2) + HashAggregate [avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_1),gid,count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_1),spark_catalog.default.store_sales.ss_list_price,propagatedFilter_1,avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_2),count(ss_list_price) FILTER (WHERE propagatedFilter_2),propagatedFilter_2,avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3),count(ss_list_price) FILTER (WHERE propagatedFilter_3),propagatedFilter_3,avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4),count(ss_list_price) FILTER (WHERE propagatedFilter_4),propagatedFilter_4,avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5),count(ss_list_price) FILTER (WHERE propagatedFilter_5),propagatedFilter_5,avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6),count(ss_list_price) FILTER (WHERE propagatedFilter_6),propagatedFilter_6] [first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count,first,valueSet,first,valueSet,count] + HashAggregate [spark_catalog.default.store_sales.ss_list_price,gid,max,max,max,max,max,max,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count] [max(propagatedFilter_1),max(propagatedFilter_2),max(propagatedFilter_3),max(propagatedFilter_4),max(propagatedFilter_5),max(propagatedFilter_6),avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)L),count(spark_catalog.default.store_sales.ss_list_price),avg(unscaledvalue(ss_list_price)L),count(ss_list_price),avg(unscaledvalue(ss_list_price)L),count(ss_list_price),avg(unscaledvalue(ss_list_price)L),count(ss_list_price),avg(unscaledvalue(ss_list_price)L),count(ss_list_price),avg(unscaledvalue(ss_list_price)L),count(ss_list_price),propagatedFilter_1,propagatedFilter_2,propagatedFilter_3,propagatedFilter_4,propagatedFilter_5,propagatedFilter_6,avg(unscaledvalue(spark_catalog.default.store_sales.ss_list_price)) FILTER (WHERE propagatedFilter_1),count(spark_catalog.default.store_sales.ss_list_price) FILTER (WHERE propagatedFilter_1),avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_2),count(ss_list_price) FILTER (WHERE propagatedFilter_2),avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_3),count(ss_list_price) FILTER (WHERE propagatedFilter_3),avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_4),count(ss_list_price) FILTER (WHERE propagatedFilter_4),avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_5),count(ss_list_price) FILTER (WHERE propagatedFilter_5),avg(unscaledvalue(ss_list_price)) FILTER (WHERE propagatedFilter_6),count(ss_list_price) FILTER (WHERE propagatedFilter_6),max,max,max,max,max,max,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count] + InputAdapter + Exchange [spark_catalog.default.store_sales.ss_list_price,gid] #2 + WholeStageCodegen (1) + HashAggregate [spark_catalog.default.store_sales.ss_list_price,gid,propagatedFilter_1,propagatedFilter_2,propagatedFilter_3,propagatedFilter_4,propagatedFilter_5,propagatedFilter_6,unscaledvalue(spark_catalog.default.store_sales.ss_list_price),propagatedFilter_1,spark_catalog.default.store_sales.ss_list_price,unscaledvalue(ss_list_price),propagatedFilter_2,ss_list_price,propagatedFilter_3,propagatedFilter_4,propagatedFilter_5,propagatedFilter_6] [max,max,max,max,max,max,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count,max,max,max,max,max,max,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count,sum,count,count] + Expand [ss_list_price,propagatedFilter_1,propagatedFilter_2,propagatedFilter_3,propagatedFilter_4,propagatedFilter_5,propagatedFilter_6] + Project [ss_list_price,ss_quantity,ss_coupon_amt,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation InputAdapter BroadcastExchange #3 - WholeStageCodegen (5) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] - InputAdapter - Exchange #4 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #5 - WholeStageCodegen (3) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + WholeStageCodegen (1) + Project + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] - InputAdapter - Exchange #7 - WholeStageCodegen (7) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #8 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastExchange #4 + WholeStageCodegen (2) + Project + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation InputAdapter - BroadcastExchange #9 - WholeStageCodegen (11) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] - InputAdapter - Exchange #10 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #11 - WholeStageCodegen (9) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastExchange #5 + WholeStageCodegen (3) + Project + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation InputAdapter - BroadcastExchange #12 - WholeStageCodegen (14) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] - InputAdapter - Exchange #13 - WholeStageCodegen (13) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #14 - WholeStageCodegen (12) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastExchange #6 + WholeStageCodegen (4) + Project + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation InputAdapter - BroadcastExchange #15 - WholeStageCodegen (17) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] - InputAdapter - Exchange #16 - WholeStageCodegen (16) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #17 - WholeStageCodegen (15) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastExchange #7 + WholeStageCodegen (5) + Project + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Scan OneRowRelation diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt index 4baef9301504c..ea2a81fb7fa6e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt @@ -82,7 +82,7 @@ Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] (6) Filter [codegen id : 1] Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : ((isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) AND might_contain(Subquery scalar-subquery#7, [id=#1], xxhash64(d_week_seq#5, 42))) +Condition : ((isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) AND might_contain(Subquery scalar-subquery#7, [id=#1].bloomFilter, xxhash64(d_week_seq#5, 42))) (7) BroadcastExchange Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] @@ -203,11 +203,11 @@ Input [3]: [d_date_sk#54, d_week_seq#55, d_day_name#56] (31) Filter [codegen id : 5] Input [3]: [d_date_sk#54, d_week_seq#55, d_day_name#56] -Condition : ((isnotnull(d_date_sk#54) AND isnotnull(d_week_seq#55)) AND might_contain(Subquery scalar-subquery#57, [id=#6], xxhash64(d_week_seq#55, 42))) +Condition : ((isnotnull(d_date_sk#54) AND isnotnull(d_week_seq#55)) AND might_contain(ReusedSubquery Subquery scalar-subquery#7, [id=#1].bloomFilter, xxhash64(d_week_seq#55, 42))) (32) BroadcastExchange Input [3]: [d_date_sk#54, d_week_seq#55, d_day_name#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#53] @@ -223,115 +223,116 @@ Input [6]: [ss_store_sk#51, ss_sales_price#52, ss_sold_date_sk#53, d_date_sk#54, Input [4]: [ss_store_sk#51, ss_sales_price#52, d_week_seq#55, d_day_name#56] Keys [2]: [d_week_seq#55, ss_store_sk#51] Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Sunday ) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Monday ) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Tuesday ) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Wednesday) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Thursday ) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Friday ) THEN ss_sales_price#52 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#56 = Saturday ) THEN ss_sales_price#52 END))] -Aggregate Attributes [7]: [sum#58, sum#59, sum#60, sum#61, sum#62, sum#63, sum#64] -Results [9]: [d_week_seq#55, ss_store_sk#51, sum#65, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] +Aggregate Attributes [7]: [sum#57, sum#58, sum#59, sum#60, sum#61, sum#62, sum#63] +Results [9]: [d_week_seq#55, ss_store_sk#51, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69, sum#70] (36) Exchange -Input [9]: [d_week_seq#55, ss_store_sk#51, sum#65, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] -Arguments: hashpartitioning(d_week_seq#55, ss_store_sk#51, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [9]: [d_week_seq#55, ss_store_sk#51, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69, sum#70] +Arguments: hashpartitioning(d_week_seq#55, ss_store_sk#51, 5), ENSURE_REQUIREMENTS, [plan_id=7] (37) HashAggregate [codegen id : 9] -Input [9]: [d_week_seq#55, ss_store_sk#51, sum#65, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] +Input [9]: [d_week_seq#55, ss_store_sk#51, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69, sum#70] Keys [2]: [d_week_seq#55, ss_store_sk#51] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#56 = Sunday ) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Monday ) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Tuesday ) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Wednesday) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Thursday ) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Friday ) THEN ss_sales_price#52 END)), sum(UnscaledValue(CASE WHEN (d_day_name#56 = Saturday ) THEN ss_sales_price#52 END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#56 = Sunday ) THEN ss_sales_price#52 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Monday ) THEN ss_sales_price#52 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Tuesday ) THEN ss_sales_price#52 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Wednesday) THEN ss_sales_price#52 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Thursday ) THEN ss_sales_price#52 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Friday ) THEN ss_sales_price#52 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#56 = Saturday ) THEN ss_sales_price#52 END))#28] -Results [9]: [d_week_seq#55, ss_store_sk#51, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Sunday ) THEN ss_sales_price#52 END))#22,17,2) AS sun_sales#72, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Monday ) THEN ss_sales_price#52 END))#23,17,2) AS mon_sales#73, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Tuesday ) THEN ss_sales_price#52 END))#24,17,2) AS tue_sales#74, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Wednesday) THEN ss_sales_price#52 END))#25,17,2) AS wed_sales#75, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Thursday ) THEN ss_sales_price#52 END))#26,17,2) AS thu_sales#76, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Friday ) THEN ss_sales_price#52 END))#27,17,2) AS fri_sales#77, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Saturday ) THEN ss_sales_price#52 END))#28,17,2) AS sat_sales#78] +Results [9]: [d_week_seq#55, ss_store_sk#51, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Sunday ) THEN ss_sales_price#52 END))#22,17,2) AS sun_sales#71, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Monday ) THEN ss_sales_price#52 END))#23,17,2) AS mon_sales#72, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Tuesday ) THEN ss_sales_price#52 END))#24,17,2) AS tue_sales#73, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Wednesday) THEN ss_sales_price#52 END))#25,17,2) AS wed_sales#74, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Thursday ) THEN ss_sales_price#52 END))#26,17,2) AS thu_sales#75, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Friday ) THEN ss_sales_price#52 END))#27,17,2) AS fri_sales#76, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#56 = Saturday ) THEN ss_sales_price#52 END))#28,17,2) AS sat_sales#77] (38) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#79, s_store_id#80] +Output [2]: [s_store_sk#78, s_store_id#79] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct (39) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#79, s_store_id#80] +Input [2]: [s_store_sk#78, s_store_id#79] (40) Filter [codegen id : 7] -Input [2]: [s_store_sk#79, s_store_id#80] -Condition : (isnotnull(s_store_sk#79) AND isnotnull(s_store_id#80)) +Input [2]: [s_store_sk#78, s_store_id#79] +Condition : (isnotnull(s_store_sk#78) AND isnotnull(s_store_id#79)) (41) BroadcastExchange -Input [2]: [s_store_sk#79, s_store_id#80] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +Input [2]: [s_store_sk#78, s_store_id#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#79] +Right keys [1]: [s_store_sk#78] Join type: Inner Join condition: None (43) Project [codegen id : 9] -Output [9]: [d_week_seq#55, sun_sales#72, mon_sales#73, tue_sales#74, wed_sales#75, thu_sales#76, fri_sales#77, sat_sales#78, s_store_id#80] -Input [11]: [d_week_seq#55, ss_store_sk#51, sun_sales#72, mon_sales#73, tue_sales#74, wed_sales#75, thu_sales#76, fri_sales#77, sat_sales#78, s_store_sk#79, s_store_id#80] +Output [9]: [d_week_seq#55, sun_sales#71, mon_sales#72, tue_sales#73, wed_sales#74, thu_sales#75, fri_sales#76, sat_sales#77, s_store_id#79] +Input [11]: [d_week_seq#55, ss_store_sk#51, sun_sales#71, mon_sales#72, tue_sales#73, wed_sales#74, thu_sales#75, fri_sales#76, sat_sales#77, s_store_sk#78, s_store_id#79] (44) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#81, d_week_seq#82] +Output [2]: [d_month_seq#80, d_week_seq#81] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] ReadSchema: struct (45) ColumnarToRow [codegen id : 8] -Input [2]: [d_month_seq#81, d_week_seq#82] +Input [2]: [d_month_seq#80, d_week_seq#81] (46) Filter [codegen id : 8] -Input [2]: [d_month_seq#81, d_week_seq#82] -Condition : (((isnotnull(d_month_seq#81) AND (d_month_seq#81 >= 1224)) AND (d_month_seq#81 <= 1235)) AND isnotnull(d_week_seq#82)) +Input [2]: [d_month_seq#80, d_week_seq#81] +Condition : (((isnotnull(d_month_seq#80) AND (d_month_seq#80 >= 1224)) AND (d_month_seq#80 <= 1235)) AND isnotnull(d_week_seq#81)) (47) Project [codegen id : 8] -Output [1]: [d_week_seq#82] -Input [2]: [d_month_seq#81, d_week_seq#82] +Output [1]: [d_week_seq#81] +Input [2]: [d_month_seq#80, d_week_seq#81] (48) BroadcastExchange -Input [1]: [d_week_seq#82] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Input [1]: [d_week_seq#81] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] (49) BroadcastHashJoin [codegen id : 9] Left keys [1]: [d_week_seq#55] -Right keys [1]: [d_week_seq#82] +Right keys [1]: [d_week_seq#81] Join type: Inner Join condition: None (50) Project [codegen id : 9] -Output [9]: [d_week_seq#55 AS d_week_seq2#83, s_store_id#80 AS s_store_id2#84, sun_sales#72 AS sun_sales2#85, mon_sales#73 AS mon_sales2#86, tue_sales#74 AS tue_sales2#87, wed_sales#75 AS wed_sales2#88, thu_sales#76 AS thu_sales2#89, fri_sales#77 AS fri_sales2#90, sat_sales#78 AS sat_sales2#91] -Input [10]: [d_week_seq#55, sun_sales#72, mon_sales#73, tue_sales#74, wed_sales#75, thu_sales#76, fri_sales#77, sat_sales#78, s_store_id#80, d_week_seq#82] +Output [9]: [d_week_seq#55 AS d_week_seq2#82, s_store_id#79 AS s_store_id2#83, sun_sales#71 AS sun_sales2#84, mon_sales#72 AS mon_sales2#85, tue_sales#73 AS tue_sales2#86, wed_sales#74 AS wed_sales2#87, thu_sales#75 AS thu_sales2#88, fri_sales#76 AS fri_sales2#89, sat_sales#77 AS sat_sales2#90] +Input [10]: [d_week_seq#55, sun_sales#71, mon_sales#72, tue_sales#73, wed_sales#74, thu_sales#75, fri_sales#76, sat_sales#77, s_store_id#79, d_week_seq#81] (51) BroadcastExchange -Input [9]: [d_week_seq2#83, s_store_id2#84, sun_sales2#85, mon_sales2#86, tue_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=11] +Input [9]: [d_week_seq2#82, s_store_id2#83, sun_sales2#84, mon_sales2#85, tue_sales2#86, wed_sales2#87, thu_sales2#88, fri_sales2#89, sat_sales2#90] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=10] (52) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#43, d_week_seq1#42] -Right keys [2]: [s_store_id2#84, (d_week_seq2#83 - 52)] +Right keys [2]: [s_store_id2#83, (d_week_seq2#82 - 52)] Join type: Inner Join condition: None (53) Project [codegen id : 10] -Output [10]: [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1#44 / sun_sales2#85) AS (sun_sales1 / sun_sales2)#92, (mon_sales1#45 / mon_sales2#86) AS (mon_sales1 / mon_sales2)#93, (tue_sales1#46 / tue_sales2#87) AS (tue_sales1 / tue_sales2)#94, (wed_sales1#47 / wed_sales2#88) AS (wed_sales1 / wed_sales2)#95, (thu_sales1#48 / thu_sales2#89) AS (thu_sales1 / thu_sales2)#96, (fri_sales1#49 / fri_sales2#90) AS (fri_sales1 / fri_sales2)#97, (sat_sales1#50 / sat_sales2#91) AS (sat_sales1 / sat_sales2)#98] -Input [19]: [s_store_name1#41, d_week_seq1#42, s_store_id1#43, sun_sales1#44, mon_sales1#45, tue_sales1#46, wed_sales1#47, thu_sales1#48, fri_sales1#49, sat_sales1#50, d_week_seq2#83, s_store_id2#84, sun_sales2#85, mon_sales2#86, tue_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91] +Output [10]: [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1#44 / sun_sales2#84) AS (sun_sales1 / sun_sales2)#91, (mon_sales1#45 / mon_sales2#85) AS (mon_sales1 / mon_sales2)#92, (tue_sales1#46 / tue_sales2#86) AS (tue_sales1 / tue_sales2)#93, (wed_sales1#47 / wed_sales2#87) AS (wed_sales1 / wed_sales2)#94, (thu_sales1#48 / thu_sales2#88) AS (thu_sales1 / thu_sales2)#95, (fri_sales1#49 / fri_sales2#89) AS (fri_sales1 / fri_sales2)#96, (sat_sales1#50 / sat_sales2#90) AS (sat_sales1 / sat_sales2)#97] +Input [19]: [s_store_name1#41, d_week_seq1#42, s_store_id1#43, sun_sales1#44, mon_sales1#45, tue_sales1#46, wed_sales1#47, thu_sales1#48, fri_sales1#49, sat_sales1#50, d_week_seq2#82, s_store_id2#83, sun_sales2#84, mon_sales2#85, tue_sales2#86, wed_sales2#87, thu_sales2#88, fri_sales2#89, sat_sales2#90] (54) TakeOrderedAndProject -Input [10]: [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1 / sun_sales2)#92, (mon_sales1 / mon_sales2)#93, (tue_sales1 / tue_sales2)#94, (wed_sales1 / wed_sales2)#95, (thu_sales1 / thu_sales2)#96, (fri_sales1 / fri_sales2)#97, (sat_sales1 / sat_sales2)#98] -Arguments: 100, [s_store_name1#41 ASC NULLS FIRST, s_store_id1#43 ASC NULLS FIRST, d_week_seq1#42 ASC NULLS FIRST], [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1 / sun_sales2)#92, (mon_sales1 / mon_sales2)#93, (tue_sales1 / tue_sales2)#94, (wed_sales1 / wed_sales2)#95, (thu_sales1 / thu_sales2)#96, (fri_sales1 / fri_sales2)#97, (sat_sales1 / sat_sales2)#98] +Input [10]: [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1 / sun_sales2)#91, (mon_sales1 / mon_sales2)#92, (tue_sales1 / tue_sales2)#93, (wed_sales1 / wed_sales2)#94, (thu_sales1 / thu_sales2)#95, (fri_sales1 / fri_sales2)#96, (sat_sales1 / sat_sales2)#97] +Arguments: 100, [s_store_name1#41 ASC NULLS FIRST, s_store_id1#43 ASC NULLS FIRST, d_week_seq1#42 ASC NULLS FIRST], [s_store_name1#41, s_store_id1#43, d_week_seq1#42, (sun_sales1 / sun_sales2)#91, (mon_sales1 / mon_sales2)#92, (tue_sales1 / tue_sales2)#93, (wed_sales1 / wed_sales2)#94, (thu_sales1 / thu_sales2)#95, (fri_sales1 / fri_sales2)#96, (sat_sales1 / sat_sales2)#97] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery#7, [id=#1] -ObjectHashAggregate (61) -+- Exchange (60) - +- ObjectHashAggregate (59) - +- * Project (58) - +- * Filter (57) - +- * ColumnarToRow (56) - +- Scan parquet spark_catalog.default.date_dim (55) +* Project (62) ++- ObjectHashAggregate (61) + +- Exchange (60) + +- ObjectHashAggregate (59) + +- * Project (58) + +- * Filter (57) + +- * ColumnarToRow (56) + +- Scan parquet spark_catalog.default.date_dim (55) (55) Scan parquet spark_catalog.default.date_dim Output [2]: [d_month_seq#39, d_week_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_week_seq), Or(And(GreaterThanOrEqual(d_month_seq,1224),LessThanOrEqual(d_month_seq,1235)),And(GreaterThanOrEqual(d_month_seq,1212),LessThanOrEqual(d_month_seq,1223)))] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] @@ -339,74 +340,34 @@ Input [2]: [d_month_seq#39, d_week_seq#40] (57) Filter [codegen id : 1] Input [2]: [d_month_seq#39, d_week_seq#40] -Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1212)) AND (d_month_seq#39 <= 1223)) AND isnotnull(d_week_seq#40)) +Condition : ((isnotnull(d_month_seq#39) AND isnotnull(d_week_seq#40)) AND (((d_month_seq#39 >= 1224) AND (d_month_seq#39 <= 1235)) OR ((d_month_seq#39 >= 1212) AND (d_month_seq#39 <= 1223)))) (58) Project [codegen id : 1] -Output [1]: [d_week_seq#40] +Output [3]: [d_week_seq#40, ((isnotnull(d_month_seq#39) AND ((d_month_seq#39 >= 1224) AND (d_month_seq#39 <= 1235))) AND isnotnull(d_week_seq#40)) AS propagatedFilter_1#98, ((isnotnull(d_month_seq#39) AND ((d_month_seq#39 >= 1212) AND (d_month_seq#39 <= 1223))) AND isnotnull(d_week_seq#40)) AS propagatedFilter_2#99] Input [2]: [d_month_seq#39, d_week_seq#40] (59) ObjectHashAggregate -Input [1]: [d_week_seq#40] +Input [3]: [d_week_seq#40, propagatedFilter_1#98, propagatedFilter_2#99] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [buf#99] -Results [1]: [buf#100] +Functions [2]: [partial_bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0) FILTER (WHERE propagatedFilter_2#99), partial_bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0) FILTER (WHERE propagatedFilter_1#98)] +Aggregate Attributes [2]: [buf#100, buf#101] +Results [2]: [buf#102, buf#103] (60) Exchange -Input [1]: [buf#100] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +Input [2]: [buf#102, buf#103] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] (61) ObjectHashAggregate -Input [1]: [buf#100] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#101] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#101 AS bloomFilter#102] - -Subquery:2 Hosting operator id = 31 Hosting Expression = Subquery scalar-subquery#57, [id=#6] -ObjectHashAggregate (68) -+- Exchange (67) - +- ObjectHashAggregate (66) - +- * Project (65) - +- * Filter (64) - +- * ColumnarToRow (63) - +- Scan parquet spark_catalog.default.date_dim (62) - - -(62) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#81, d_week_seq#82] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(63) ColumnarToRow [codegen id : 1] -Input [2]: [d_month_seq#81, d_week_seq#82] - -(64) Filter [codegen id : 1] -Input [2]: [d_month_seq#81, d_week_seq#82] -Condition : (((isnotnull(d_month_seq#81) AND (d_month_seq#81 >= 1224)) AND (d_month_seq#81 <= 1235)) AND isnotnull(d_week_seq#82)) - -(65) Project [codegen id : 1] -Output [1]: [d_week_seq#82] -Input [2]: [d_month_seq#81, d_week_seq#82] - -(66) ObjectHashAggregate -Input [1]: [d_week_seq#82] +Input [2]: [buf#102, buf#103] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#82, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [buf#103] -Results [1]: [buf#104] +Functions [2]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0), bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)] +Aggregate Attributes [2]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#104, bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#105] +Results [2]: [bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#104 AS bloomFilter#106, bloom_filter_agg(xxhash64(d_week_seq#40, 42), 335, 8990, 0, 0)#105 AS bloomFilter#107] -(67) Exchange -Input [1]: [buf#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(62) Project [codegen id : 2] +Output [1]: [named_struct(bloomFilter, bloomFilter#106, bloomFilter, bloomFilter#107) AS mergedValue#108] +Input [2]: [bloomFilter#106, bloomFilter#107] -(68) ObjectHashAggregate -Input [1]: [buf#104] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#82, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#82, 42), 335, 8990, 0, 0)#105] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#82, 42), 335, 8990, 0, 0)#105 AS bloomFilter#106] +Subquery:2 Hosting operator id = 31 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#1] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt index 62f4fab4891e0..d0e722152ba7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt @@ -22,15 +22,18 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s WholeStageCodegen (1) Filter [d_date_sk,d_week_seq] Subquery #1 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 335, 8990, 0, 0),bloomFilter,buf] - Exchange #3 - ObjectHashAggregate [d_week_seq] [buf,buf] - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_month_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + WholeStageCodegen (2) + Project [bloomFilter,bloomFilter] + InputAdapter + ObjectHashAggregate [buf,buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 335, 8990, 0, 0),bloom_filter_agg(xxhash64(d_week_seq, 42), 335, 8990, 0, 0),bloomFilter,bloomFilter,buf,buf] + Exchange #3 + ObjectHashAggregate [d_week_seq,propagatedFilter_1,propagatedFilter_2] [buf,buf,buf,buf] + WholeStageCodegen (1) + Project [d_week_seq,d_month_seq] + Filter [d_month_seq,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] @@ -71,28 +74,19 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s BroadcastExchange #8 WholeStageCodegen (5) Filter [d_date_sk,d_week_seq] - Subquery #2 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 335, 8990, 0, 0),bloomFilter,buf] - Exchange #9 - ObjectHashAggregate [d_week_seq] [buf,buf] - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_month_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + ReusedSubquery [mergedValue] #1 ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (7) Filter [s_store_sk,s_store_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter - BroadcastExchange #11 + BroadcastExchange #10 WholeStageCodegen (8) Project [d_week_seq] Filter [d_month_seq,d_week_seq] 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 b789c51891fc3..42cbe546f6eb8 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,7 +20,7 @@ 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=#1].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket1#3, CASE WHEN (Subquery scalar-subquery#4, [id=#2].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#4, [id=#2].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#4, [id=#2].avg(ss_net_paid) END AS bucket2#5, CASE WHEN (Subquery scalar-subquery#6, [id=#3].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#6, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#6, [id=#3].avg(ss_net_paid) END AS bucket3#7, CASE WHEN (Subquery scalar-subquery#8, [id=#4].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#4].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#4].avg(ss_net_paid) END AS bucket4#9, CASE WHEN (Subquery scalar-subquery#10, [id=#5].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#10, [id=#5].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#10, [id=#5].avg(ss_net_paid) END AS bucket5#11] +Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#1].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket1#3, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#1].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket2#4, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#1].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket3#5, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#1].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket4#6, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#1].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket5#7] Input [1]: [r_reason_sk#1] ===== Subqueries ===== @@ -37,267 +37,71 @@ Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#12, ss_ext_discount_amt#13, ss_net_paid#14, ss_sold_date_sk#15] +Output [4]: [ss_quantity#8, ss_ext_discount_amt#9, ss_net_paid#10, ss_sold_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] +PushedFilters: [IsNotNull(ss_quantity), Or(Or(Or(Or(And(GreaterThanOrEqual(ss_quantity,21),LessThanOrEqual(ss_quantity,40)),And(GreaterThanOrEqual(ss_quantity,1),LessThanOrEqual(ss_quantity,20))),And(GreaterThanOrEqual(ss_quantity,41),LessThanOrEqual(ss_quantity,60))),And(GreaterThanOrEqual(ss_quantity,61),LessThanOrEqual(ss_quantity,80))),And(GreaterThanOrEqual(ss_quantity,81),LessThanOrEqual(ss_quantity,100)))] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#12, ss_ext_discount_amt#13, ss_net_paid#14, ss_sold_date_sk#15] +Input [4]: [ss_quantity#8, ss_ext_discount_amt#9, ss_net_paid#10, ss_sold_date_sk#11] (7) Filter [codegen id : 1] -Input [4]: [ss_quantity#12, ss_ext_discount_amt#13, ss_net_paid#14, ss_sold_date_sk#15] -Condition : ((isnotnull(ss_quantity#12) AND (ss_quantity#12 >= 1)) AND (ss_quantity#12 <= 20)) +Input [4]: [ss_quantity#8, ss_ext_discount_amt#9, ss_net_paid#10, ss_sold_date_sk#11] +Condition : (isnotnull(ss_quantity#8) AND ((((((ss_quantity#8 >= 21) AND (ss_quantity#8 <= 40)) OR ((ss_quantity#8 >= 1) AND (ss_quantity#8 <= 20))) OR ((ss_quantity#8 >= 41) AND (ss_quantity#8 <= 60))) OR ((ss_quantity#8 >= 61) AND (ss_quantity#8 <= 80))) OR ((ss_quantity#8 >= 81) AND (ss_quantity#8 <= 100)))) (8) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#13, ss_net_paid#14] -Input [4]: [ss_quantity#12, ss_ext_discount_amt#13, ss_net_paid#14, ss_sold_date_sk#15] +Output [7]: [ss_ext_discount_amt#9, ss_net_paid#10, (isnotnull(ss_quantity#8) AND ((ss_quantity#8 >= 21) AND (ss_quantity#8 <= 40))) AS propagatedFilter_1#12, (isnotnull(ss_quantity#8) AND ((ss_quantity#8 >= 1) AND (ss_quantity#8 <= 20))) AS propagatedFilter_2#13, (isnotnull(ss_quantity#8) AND ((ss_quantity#8 >= 41) AND (ss_quantity#8 <= 60))) AS propagatedFilter_3#14, (isnotnull(ss_quantity#8) AND ((ss_quantity#8 >= 61) AND (ss_quantity#8 <= 80))) AS propagatedFilter_4#15, (isnotnull(ss_quantity#8) AND ((ss_quantity#8 >= 81) AND (ss_quantity#8 <= 100))) AS propagatedFilter_5#16] +Input [4]: [ss_quantity#8, ss_ext_discount_amt#9, ss_net_paid#10, ss_sold_date_sk#11] (9) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#13, ss_net_paid#14] +Input [7]: [ss_ext_discount_amt#9, ss_net_paid#10, propagatedFilter_1#12, propagatedFilter_2#13, propagatedFilter_3#14, propagatedFilter_4#15, propagatedFilter_5#16] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#13)), partial_avg(UnscaledValue(ss_net_paid#14))] -Aggregate Attributes [5]: [count#16, sum#17, count#18, sum#19, count#20] -Results [5]: [count#21, sum#22, count#23, sum#24, count#25] +Functions [15]: [partial_count(1) FILTER (WHERE propagatedFilter_2#13), partial_avg(UnscaledValue(ss_ext_discount_amt#9)) FILTER (WHERE propagatedFilter_2#13), partial_avg(UnscaledValue(ss_net_paid#10)) FILTER (WHERE propagatedFilter_2#13), partial_count(1) FILTER (WHERE propagatedFilter_1#12), partial_avg(UnscaledValue(ss_ext_discount_amt#9)) FILTER (WHERE propagatedFilter_1#12), partial_avg(UnscaledValue(ss_net_paid#10)) FILTER (WHERE propagatedFilter_1#12), partial_count(1) FILTER (WHERE propagatedFilter_3#14), partial_avg(UnscaledValue(ss_ext_discount_amt#9)) FILTER (WHERE propagatedFilter_3#14), partial_avg(UnscaledValue(ss_net_paid#10)) FILTER (WHERE propagatedFilter_3#14), partial_count(1) FILTER (WHERE propagatedFilter_4#15), partial_avg(UnscaledValue(ss_ext_discount_amt#9)) FILTER (WHERE propagatedFilter_4#15), partial_avg(UnscaledValue(ss_net_paid#10)) FILTER (WHERE propagatedFilter_4#15), partial_count(1) FILTER (WHERE propagatedFilter_5#16), partial_avg(UnscaledValue(ss_ext_discount_amt#9)) FILTER (WHERE propagatedFilter_5#16), partial_avg(UnscaledValue(ss_net_paid#10)) FILTER (WHERE propagatedFilter_5#16)] +Aggregate Attributes [25]: [count#17, sum#18, count#19, sum#20, count#21, count#22, sum#23, count#24, sum#25, count#26, count#27, sum#28, count#29, sum#30, count#31, count#32, sum#33, count#34, sum#35, count#36, count#37, sum#38, count#39, sum#40, count#41] +Results [25]: [count#42, sum#43, count#44, sum#45, count#46, count#47, sum#48, count#49, sum#50, count#51, count#52, sum#53, count#54, sum#55, count#56, count#57, sum#58, count#59, sum#60, count#61, count#62, sum#63, count#64, sum#65, count#66] (10) Exchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Input [25]: [count#42, sum#43, count#44, sum#45, count#46, count#47, sum#48, count#49, sum#50, count#51, count#52, sum#53, count#54, sum#55, count#56, count#57, sum#58, count#59, sum#60, count#61, count#62, sum#63, count#64, sum#65, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] (11) HashAggregate [codegen id : 2] -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] +Input [25]: [count#42, sum#43, count#44, sum#45, count#46, count#47, sum#48, count#49, sum#50, count#51, count#52, sum#53, count#54, sum#55, count#56, count#57, sum#58, count#59, sum#60, count#61, count#62, sum#63, count#64, sum#65, count#66] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#13)), avg(UnscaledValue(ss_net_paid#14))] -Aggregate Attributes [3]: [count(1)#26, avg(UnscaledValue(ss_ext_discount_amt#13))#27, avg(UnscaledValue(ss_net_paid#14))#28] -Results [3]: [count(1)#26 AS count(1)#29, cast((avg(UnscaledValue(ss_ext_discount_amt#13))#27 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#30, cast((avg(UnscaledValue(ss_net_paid#14))#28 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#31] +Functions [15]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#9)), avg(UnscaledValue(ss_net_paid#10)), count(1), avg(UnscaledValue(ss_ext_discount_amt#9)), avg(UnscaledValue(ss_net_paid#10)), count(1), avg(UnscaledValue(ss_ext_discount_amt#9)), avg(UnscaledValue(ss_net_paid#10)), count(1), avg(UnscaledValue(ss_ext_discount_amt#9)), avg(UnscaledValue(ss_net_paid#10)), count(1), avg(UnscaledValue(ss_ext_discount_amt#9)), avg(UnscaledValue(ss_net_paid#10))] +Aggregate Attributes [15]: [count(1)#67, avg(UnscaledValue(ss_ext_discount_amt#9))#68, avg(UnscaledValue(ss_net_paid#10))#69, count(1)#70, avg(UnscaledValue(ss_ext_discount_amt#9))#71, avg(UnscaledValue(ss_net_paid#10))#72, count(1)#73, avg(UnscaledValue(ss_ext_discount_amt#9))#74, avg(UnscaledValue(ss_net_paid#10))#75, count(1)#76, avg(UnscaledValue(ss_ext_discount_amt#9))#77, avg(UnscaledValue(ss_net_paid#10))#78, count(1)#79, avg(UnscaledValue(ss_ext_discount_amt#9))#80, avg(UnscaledValue(ss_net_paid#10))#81] +Results [15]: [count(1)#67 AS count(1)#82, cast((avg(UnscaledValue(ss_ext_discount_amt#9))#68 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#83, cast((avg(UnscaledValue(ss_net_paid#10))#69 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#84, count(1)#70 AS count(1)#85, cast((avg(UnscaledValue(ss_ext_discount_amt#9))#71 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#86, cast((avg(UnscaledValue(ss_net_paid#10))#72 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#87, count(1)#73 AS count(1)#88, cast((avg(UnscaledValue(ss_ext_discount_amt#9))#74 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#89, cast((avg(UnscaledValue(ss_net_paid#10))#75 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90, count(1)#76 AS count(1)#91, cast((avg(UnscaledValue(ss_ext_discount_amt#9))#77 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#92, cast((avg(UnscaledValue(ss_net_paid#10))#78 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#93, count(1)#79 AS count(1)#94, cast((avg(UnscaledValue(ss_ext_discount_amt#9))#80 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#95, cast((avg(UnscaledValue(ss_net_paid#10))#81 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#96] (12) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#29, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#30, avg(ss_net_paid), avg(ss_net_paid)#31) AS mergedValue#32] -Input [3]: [count(1)#29, avg(ss_ext_discount_amt)#30, avg(ss_net_paid)#31] +Output [1]: [named_struct(count(1), count(1)#82, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#83, avg(ss_net_paid), avg(ss_net_paid)#84, count(1), count(1)#85, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#86, avg(ss_net_paid), avg(ss_net_paid)#87, count(1), count(1)#88, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#89, avg(ss_net_paid), avg(ss_net_paid)#90, count(1), count(1)#91, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#92, avg(ss_net_paid), avg(ss_net_paid)#93, count(1), count(1)#94, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#95, avg(ss_net_paid), avg(ss_net_paid)#96) AS mergedValue#97] +Input [15]: [count(1)#82, avg(ss_ext_discount_amt)#83, avg(ss_net_paid)#84, count(1)#85, avg(ss_ext_discount_amt)#86, avg(ss_net_paid)#87, count(1)#88, avg(ss_ext_discount_amt)#89, avg(ss_net_paid)#90, count(1)#91, avg(ss_ext_discount_amt)#92, avg(ss_net_paid)#93, count(1)#94, avg(ss_ext_discount_amt)#95, avg(ss_net_paid)#96] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#4, [id=#2] -* Project (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * Filter (15) - +- * ColumnarToRow (14) - +- Scan parquet spark_catalog.default.store_sales (13) +Subquery:4 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] +Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -(13) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] -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 - -(14) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] - -(15) Filter [codegen id : 1] -Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] -Condition : ((isnotnull(ss_quantity#33) AND (ss_quantity#33 >= 21)) AND (ss_quantity#33 <= 40)) - -(16) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#34, ss_net_paid#35] -Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] - -(17) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#34, ss_net_paid#35] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#34)), partial_avg(UnscaledValue(ss_net_paid#35))] -Aggregate Attributes [5]: [count#37, sum#38, count#39, sum#40, count#41] -Results [5]: [count#42, sum#43, count#44, sum#45, count#46] - -(18) Exchange -Input [5]: [count#42, sum#43, count#44, sum#45, count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] - -(19) HashAggregate [codegen id : 2] -Input [5]: [count#42, sum#43, count#44, sum#45, count#46] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#34)), avg(UnscaledValue(ss_net_paid#35))] -Aggregate Attributes [3]: [count(1)#47, avg(UnscaledValue(ss_ext_discount_amt#34))#48, avg(UnscaledValue(ss_net_paid#35))#49] -Results [3]: [count(1)#47 AS count(1)#50, cast((avg(UnscaledValue(ss_ext_discount_amt#34))#48 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51, cast((avg(UnscaledValue(ss_net_paid#35))#49 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#52] - -(20) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#50, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#51, avg(ss_net_paid), avg(ss_net_paid)#52) AS mergedValue#53] -Input [3]: [count(1)#50, avg(ss_ext_discount_amt)#51, avg(ss_net_paid)#52] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#4, [id=#2] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#4, [id=#2] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#3] -* Project (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * Filter (23) - +- * ColumnarToRow (22) - +- Scan parquet spark_catalog.default.store_sales (21) - - -(21) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#54, ss_ext_discount_amt#55, ss_net_paid#56, ss_sold_date_sk#57] -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 - -(22) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#54, ss_ext_discount_amt#55, ss_net_paid#56, ss_sold_date_sk#57] - -(23) Filter [codegen id : 1] -Input [4]: [ss_quantity#54, ss_ext_discount_amt#55, ss_net_paid#56, ss_sold_date_sk#57] -Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 41)) AND (ss_quantity#54 <= 60)) - -(24) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#55, ss_net_paid#56] -Input [4]: [ss_quantity#54, ss_ext_discount_amt#55, ss_net_paid#56, ss_sold_date_sk#57] - -(25) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#55, ss_net_paid#56] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#55)), partial_avg(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [5]: [count#58, sum#59, count#60, sum#61, count#62] -Results [5]: [count#63, sum#64, count#65, sum#66, count#67] - -(26) Exchange -Input [5]: [count#63, sum#64, count#65, sum#66, count#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] - -(27) HashAggregate [codegen id : 2] -Input [5]: [count#63, sum#64, count#65, sum#66, count#67] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#55)), avg(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [3]: [count(1)#68, avg(UnscaledValue(ss_ext_discount_amt#55))#69, avg(UnscaledValue(ss_net_paid#56))#70] -Results [3]: [count(1)#68 AS count(1)#71, cast((avg(UnscaledValue(ss_ext_discount_amt#55))#69 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#72, cast((avg(UnscaledValue(ss_net_paid#56))#70 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#73] +Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -(28) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#71, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#72, avg(ss_net_paid), avg(ss_net_paid)#73) AS mergedValue#74] -Input [3]: [count(1)#71, avg(ss_ext_discount_amt)#72, avg(ss_net_paid)#73] +Subquery:7 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#6, [id=#3] +Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#6, [id=#3] +Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#4] -* Project (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet spark_catalog.default.store_sales (29) +Subquery:10 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] +Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -(29) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#75, ss_ext_discount_amt#76, ss_net_paid#77, ss_sold_date_sk#78] -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 - -(30) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#75, ss_ext_discount_amt#76, ss_net_paid#77, ss_sold_date_sk#78] - -(31) Filter [codegen id : 1] -Input [4]: [ss_quantity#75, ss_ext_discount_amt#76, ss_net_paid#77, ss_sold_date_sk#78] -Condition : ((isnotnull(ss_quantity#75) AND (ss_quantity#75 >= 61)) AND (ss_quantity#75 <= 80)) - -(32) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#76, ss_net_paid#77] -Input [4]: [ss_quantity#75, ss_ext_discount_amt#76, ss_net_paid#77, ss_sold_date_sk#78] - -(33) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#76, ss_net_paid#77] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#76)), partial_avg(UnscaledValue(ss_net_paid#77))] -Aggregate Attributes [5]: [count#79, sum#80, count#81, sum#82, count#83] -Results [5]: [count#84, sum#85, count#86, sum#87, count#88] - -(34) Exchange -Input [5]: [count#84, sum#85, count#86, sum#87, count#88] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] - -(35) HashAggregate [codegen id : 2] -Input [5]: [count#84, sum#85, count#86, sum#87, count#88] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#76)), avg(UnscaledValue(ss_net_paid#77))] -Aggregate Attributes [3]: [count(1)#89, avg(UnscaledValue(ss_ext_discount_amt#76))#90, avg(UnscaledValue(ss_net_paid#77))#91] -Results [3]: [count(1)#89 AS count(1)#92, cast((avg(UnscaledValue(ss_ext_discount_amt#76))#90 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#93, cast((avg(UnscaledValue(ss_net_paid#77))#91 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#94] - -(36) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#92, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#93, avg(ss_net_paid), avg(ss_net_paid)#94) AS mergedValue#95] -Input [3]: [count(1)#92, avg(ss_ext_discount_amt)#93, avg(ss_net_paid)#94] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#4] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#4] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#10, [id=#5] -* Project (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet spark_catalog.default.store_sales (37) - - -(37) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#96, ss_ext_discount_amt#97, ss_net_paid#98, ss_sold_date_sk#99] -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 - -(38) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#96, ss_ext_discount_amt#97, ss_net_paid#98, ss_sold_date_sk#99] - -(39) Filter [codegen id : 1] -Input [4]: [ss_quantity#96, ss_ext_discount_amt#97, ss_net_paid#98, ss_sold_date_sk#99] -Condition : ((isnotnull(ss_quantity#96) AND (ss_quantity#96 >= 81)) AND (ss_quantity#96 <= 100)) - -(40) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#97, ss_net_paid#98] -Input [4]: [ss_quantity#96, ss_ext_discount_amt#97, ss_net_paid#98, ss_sold_date_sk#99] - -(41) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#97, ss_net_paid#98] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#97)), partial_avg(UnscaledValue(ss_net_paid#98))] -Aggregate Attributes [5]: [count#100, sum#101, count#102, sum#103, count#104] -Results [5]: [count#105, sum#106, count#107, sum#108, count#109] - -(42) Exchange -Input [5]: [count#105, sum#106, count#107, sum#108, count#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] - -(43) HashAggregate [codegen id : 2] -Input [5]: [count#105, sum#106, count#107, sum#108, count#109] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#97)), avg(UnscaledValue(ss_net_paid#98))] -Aggregate Attributes [3]: [count(1)#110, avg(UnscaledValue(ss_ext_discount_amt#97))#111, avg(UnscaledValue(ss_net_paid#98))#112] -Results [3]: [count(1)#110 AS count(1)#113, cast((avg(UnscaledValue(ss_ext_discount_amt#97))#111 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#114, cast((avg(UnscaledValue(ss_net_paid#98))#112 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#115] +Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -(44) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#113, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#114, avg(ss_net_paid), avg(ss_net_paid)#115) AS mergedValue#116] -Input [3]: [count(1)#113, avg(ss_ext_discount_amt)#114, avg(ss_net_paid)#115] +Subquery:13 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#5] +Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#5] +Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] 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 817fb0007f343..64692b9fa1cb3 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,79 +2,31 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (2) - 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] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),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(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,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] + HashAggregate [propagatedFilter_1,ss_ext_discount_amt,ss_net_paid,propagatedFilter_2,propagatedFilter_3,propagatedFilter_4,propagatedFilter_5] [count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid,ss_quantity] Filter [ss_quantity] ColumnarToRow InputAdapter Scan parquet spark_catalog.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) - 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 spark_catalog.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) - 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 spark_catalog.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) - 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 spark_catalog.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) - 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 spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - ReusedSubquery [mergedValue] #5 - ReusedSubquery [mergedValue] #5 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 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 b789c51891fc3..42cbe546f6eb8 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,7 +20,7 @@ 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=#1].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket1#3, CASE WHEN (Subquery scalar-subquery#4, [id=#2].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#4, [id=#2].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#4, [id=#2].avg(ss_net_paid) END AS bucket2#5, CASE WHEN (Subquery scalar-subquery#6, [id=#3].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#6, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#6, [id=#3].avg(ss_net_paid) END AS bucket3#7, CASE WHEN (Subquery scalar-subquery#8, [id=#4].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#4].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#4].avg(ss_net_paid) END AS bucket4#9, CASE WHEN (Subquery scalar-subquery#10, [id=#5].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#10, [id=#5].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#10, [id=#5].avg(ss_net_paid) END AS bucket5#11] +Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#1].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket1#3, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#1].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket2#4, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#1].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket3#5, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#1].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket4#6, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#1].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#1].avg(ss_net_paid) END AS bucket5#7] Input [1]: [r_reason_sk#1] ===== Subqueries ===== @@ -37,267 +37,71 @@ Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#12, ss_ext_discount_amt#13, ss_net_paid#14, ss_sold_date_sk#15] +Output [4]: [ss_quantity#8, ss_ext_discount_amt#9, ss_net_paid#10, ss_sold_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] +PushedFilters: [IsNotNull(ss_quantity), Or(Or(Or(Or(And(GreaterThanOrEqual(ss_quantity,21),LessThanOrEqual(ss_quantity,40)),And(GreaterThanOrEqual(ss_quantity,1),LessThanOrEqual(ss_quantity,20))),And(GreaterThanOrEqual(ss_quantity,41),LessThanOrEqual(ss_quantity,60))),And(GreaterThanOrEqual(ss_quantity,61),LessThanOrEqual(ss_quantity,80))),And(GreaterThanOrEqual(ss_quantity,81),LessThanOrEqual(ss_quantity,100)))] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#12, ss_ext_discount_amt#13, ss_net_paid#14, ss_sold_date_sk#15] +Input [4]: [ss_quantity#8, ss_ext_discount_amt#9, ss_net_paid#10, ss_sold_date_sk#11] (7) Filter [codegen id : 1] -Input [4]: [ss_quantity#12, ss_ext_discount_amt#13, ss_net_paid#14, ss_sold_date_sk#15] -Condition : ((isnotnull(ss_quantity#12) AND (ss_quantity#12 >= 1)) AND (ss_quantity#12 <= 20)) +Input [4]: [ss_quantity#8, ss_ext_discount_amt#9, ss_net_paid#10, ss_sold_date_sk#11] +Condition : (isnotnull(ss_quantity#8) AND ((((((ss_quantity#8 >= 21) AND (ss_quantity#8 <= 40)) OR ((ss_quantity#8 >= 1) AND (ss_quantity#8 <= 20))) OR ((ss_quantity#8 >= 41) AND (ss_quantity#8 <= 60))) OR ((ss_quantity#8 >= 61) AND (ss_quantity#8 <= 80))) OR ((ss_quantity#8 >= 81) AND (ss_quantity#8 <= 100)))) (8) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#13, ss_net_paid#14] -Input [4]: [ss_quantity#12, ss_ext_discount_amt#13, ss_net_paid#14, ss_sold_date_sk#15] +Output [7]: [ss_ext_discount_amt#9, ss_net_paid#10, (isnotnull(ss_quantity#8) AND ((ss_quantity#8 >= 21) AND (ss_quantity#8 <= 40))) AS propagatedFilter_1#12, (isnotnull(ss_quantity#8) AND ((ss_quantity#8 >= 1) AND (ss_quantity#8 <= 20))) AS propagatedFilter_2#13, (isnotnull(ss_quantity#8) AND ((ss_quantity#8 >= 41) AND (ss_quantity#8 <= 60))) AS propagatedFilter_3#14, (isnotnull(ss_quantity#8) AND ((ss_quantity#8 >= 61) AND (ss_quantity#8 <= 80))) AS propagatedFilter_4#15, (isnotnull(ss_quantity#8) AND ((ss_quantity#8 >= 81) AND (ss_quantity#8 <= 100))) AS propagatedFilter_5#16] +Input [4]: [ss_quantity#8, ss_ext_discount_amt#9, ss_net_paid#10, ss_sold_date_sk#11] (9) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#13, ss_net_paid#14] +Input [7]: [ss_ext_discount_amt#9, ss_net_paid#10, propagatedFilter_1#12, propagatedFilter_2#13, propagatedFilter_3#14, propagatedFilter_4#15, propagatedFilter_5#16] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#13)), partial_avg(UnscaledValue(ss_net_paid#14))] -Aggregate Attributes [5]: [count#16, sum#17, count#18, sum#19, count#20] -Results [5]: [count#21, sum#22, count#23, sum#24, count#25] +Functions [15]: [partial_count(1) FILTER (WHERE propagatedFilter_2#13), partial_avg(UnscaledValue(ss_ext_discount_amt#9)) FILTER (WHERE propagatedFilter_2#13), partial_avg(UnscaledValue(ss_net_paid#10)) FILTER (WHERE propagatedFilter_2#13), partial_count(1) FILTER (WHERE propagatedFilter_1#12), partial_avg(UnscaledValue(ss_ext_discount_amt#9)) FILTER (WHERE propagatedFilter_1#12), partial_avg(UnscaledValue(ss_net_paid#10)) FILTER (WHERE propagatedFilter_1#12), partial_count(1) FILTER (WHERE propagatedFilter_3#14), partial_avg(UnscaledValue(ss_ext_discount_amt#9)) FILTER (WHERE propagatedFilter_3#14), partial_avg(UnscaledValue(ss_net_paid#10)) FILTER (WHERE propagatedFilter_3#14), partial_count(1) FILTER (WHERE propagatedFilter_4#15), partial_avg(UnscaledValue(ss_ext_discount_amt#9)) FILTER (WHERE propagatedFilter_4#15), partial_avg(UnscaledValue(ss_net_paid#10)) FILTER (WHERE propagatedFilter_4#15), partial_count(1) FILTER (WHERE propagatedFilter_5#16), partial_avg(UnscaledValue(ss_ext_discount_amt#9)) FILTER (WHERE propagatedFilter_5#16), partial_avg(UnscaledValue(ss_net_paid#10)) FILTER (WHERE propagatedFilter_5#16)] +Aggregate Attributes [25]: [count#17, sum#18, count#19, sum#20, count#21, count#22, sum#23, count#24, sum#25, count#26, count#27, sum#28, count#29, sum#30, count#31, count#32, sum#33, count#34, sum#35, count#36, count#37, sum#38, count#39, sum#40, count#41] +Results [25]: [count#42, sum#43, count#44, sum#45, count#46, count#47, sum#48, count#49, sum#50, count#51, count#52, sum#53, count#54, sum#55, count#56, count#57, sum#58, count#59, sum#60, count#61, count#62, sum#63, count#64, sum#65, count#66] (10) Exchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Input [25]: [count#42, sum#43, count#44, sum#45, count#46, count#47, sum#48, count#49, sum#50, count#51, count#52, sum#53, count#54, sum#55, count#56, count#57, sum#58, count#59, sum#60, count#61, count#62, sum#63, count#64, sum#65, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] (11) HashAggregate [codegen id : 2] -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] +Input [25]: [count#42, sum#43, count#44, sum#45, count#46, count#47, sum#48, count#49, sum#50, count#51, count#52, sum#53, count#54, sum#55, count#56, count#57, sum#58, count#59, sum#60, count#61, count#62, sum#63, count#64, sum#65, count#66] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#13)), avg(UnscaledValue(ss_net_paid#14))] -Aggregate Attributes [3]: [count(1)#26, avg(UnscaledValue(ss_ext_discount_amt#13))#27, avg(UnscaledValue(ss_net_paid#14))#28] -Results [3]: [count(1)#26 AS count(1)#29, cast((avg(UnscaledValue(ss_ext_discount_amt#13))#27 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#30, cast((avg(UnscaledValue(ss_net_paid#14))#28 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#31] +Functions [15]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#9)), avg(UnscaledValue(ss_net_paid#10)), count(1), avg(UnscaledValue(ss_ext_discount_amt#9)), avg(UnscaledValue(ss_net_paid#10)), count(1), avg(UnscaledValue(ss_ext_discount_amt#9)), avg(UnscaledValue(ss_net_paid#10)), count(1), avg(UnscaledValue(ss_ext_discount_amt#9)), avg(UnscaledValue(ss_net_paid#10)), count(1), avg(UnscaledValue(ss_ext_discount_amt#9)), avg(UnscaledValue(ss_net_paid#10))] +Aggregate Attributes [15]: [count(1)#67, avg(UnscaledValue(ss_ext_discount_amt#9))#68, avg(UnscaledValue(ss_net_paid#10))#69, count(1)#70, avg(UnscaledValue(ss_ext_discount_amt#9))#71, avg(UnscaledValue(ss_net_paid#10))#72, count(1)#73, avg(UnscaledValue(ss_ext_discount_amt#9))#74, avg(UnscaledValue(ss_net_paid#10))#75, count(1)#76, avg(UnscaledValue(ss_ext_discount_amt#9))#77, avg(UnscaledValue(ss_net_paid#10))#78, count(1)#79, avg(UnscaledValue(ss_ext_discount_amt#9))#80, avg(UnscaledValue(ss_net_paid#10))#81] +Results [15]: [count(1)#67 AS count(1)#82, cast((avg(UnscaledValue(ss_ext_discount_amt#9))#68 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#83, cast((avg(UnscaledValue(ss_net_paid#10))#69 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#84, count(1)#70 AS count(1)#85, cast((avg(UnscaledValue(ss_ext_discount_amt#9))#71 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#86, cast((avg(UnscaledValue(ss_net_paid#10))#72 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#87, count(1)#73 AS count(1)#88, cast((avg(UnscaledValue(ss_ext_discount_amt#9))#74 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#89, cast((avg(UnscaledValue(ss_net_paid#10))#75 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90, count(1)#76 AS count(1)#91, cast((avg(UnscaledValue(ss_ext_discount_amt#9))#77 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#92, cast((avg(UnscaledValue(ss_net_paid#10))#78 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#93, count(1)#79 AS count(1)#94, cast((avg(UnscaledValue(ss_ext_discount_amt#9))#80 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#95, cast((avg(UnscaledValue(ss_net_paid#10))#81 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#96] (12) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#29, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#30, avg(ss_net_paid), avg(ss_net_paid)#31) AS mergedValue#32] -Input [3]: [count(1)#29, avg(ss_ext_discount_amt)#30, avg(ss_net_paid)#31] +Output [1]: [named_struct(count(1), count(1)#82, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#83, avg(ss_net_paid), avg(ss_net_paid)#84, count(1), count(1)#85, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#86, avg(ss_net_paid), avg(ss_net_paid)#87, count(1), count(1)#88, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#89, avg(ss_net_paid), avg(ss_net_paid)#90, count(1), count(1)#91, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#92, avg(ss_net_paid), avg(ss_net_paid)#93, count(1), count(1)#94, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#95, avg(ss_net_paid), avg(ss_net_paid)#96) AS mergedValue#97] +Input [15]: [count(1)#82, avg(ss_ext_discount_amt)#83, avg(ss_net_paid)#84, count(1)#85, avg(ss_ext_discount_amt)#86, avg(ss_net_paid)#87, count(1)#88, avg(ss_ext_discount_amt)#89, avg(ss_net_paid)#90, count(1)#91, avg(ss_ext_discount_amt)#92, avg(ss_net_paid)#93, count(1)#94, avg(ss_ext_discount_amt)#95, avg(ss_net_paid)#96] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#4, [id=#2] -* Project (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * Filter (15) - +- * ColumnarToRow (14) - +- Scan parquet spark_catalog.default.store_sales (13) +Subquery:4 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] +Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -(13) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] -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 - -(14) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] - -(15) Filter [codegen id : 1] -Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] -Condition : ((isnotnull(ss_quantity#33) AND (ss_quantity#33 >= 21)) AND (ss_quantity#33 <= 40)) - -(16) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#34, ss_net_paid#35] -Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] - -(17) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#34, ss_net_paid#35] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#34)), partial_avg(UnscaledValue(ss_net_paid#35))] -Aggregate Attributes [5]: [count#37, sum#38, count#39, sum#40, count#41] -Results [5]: [count#42, sum#43, count#44, sum#45, count#46] - -(18) Exchange -Input [5]: [count#42, sum#43, count#44, sum#45, count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] - -(19) HashAggregate [codegen id : 2] -Input [5]: [count#42, sum#43, count#44, sum#45, count#46] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#34)), avg(UnscaledValue(ss_net_paid#35))] -Aggregate Attributes [3]: [count(1)#47, avg(UnscaledValue(ss_ext_discount_amt#34))#48, avg(UnscaledValue(ss_net_paid#35))#49] -Results [3]: [count(1)#47 AS count(1)#50, cast((avg(UnscaledValue(ss_ext_discount_amt#34))#48 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51, cast((avg(UnscaledValue(ss_net_paid#35))#49 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#52] - -(20) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#50, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#51, avg(ss_net_paid), avg(ss_net_paid)#52) AS mergedValue#53] -Input [3]: [count(1)#50, avg(ss_ext_discount_amt)#51, avg(ss_net_paid)#52] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#4, [id=#2] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#4, [id=#2] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#3] -* Project (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * Filter (23) - +- * ColumnarToRow (22) - +- Scan parquet spark_catalog.default.store_sales (21) - - -(21) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#54, ss_ext_discount_amt#55, ss_net_paid#56, ss_sold_date_sk#57] -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 - -(22) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#54, ss_ext_discount_amt#55, ss_net_paid#56, ss_sold_date_sk#57] - -(23) Filter [codegen id : 1] -Input [4]: [ss_quantity#54, ss_ext_discount_amt#55, ss_net_paid#56, ss_sold_date_sk#57] -Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 41)) AND (ss_quantity#54 <= 60)) - -(24) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#55, ss_net_paid#56] -Input [4]: [ss_quantity#54, ss_ext_discount_amt#55, ss_net_paid#56, ss_sold_date_sk#57] - -(25) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#55, ss_net_paid#56] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#55)), partial_avg(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [5]: [count#58, sum#59, count#60, sum#61, count#62] -Results [5]: [count#63, sum#64, count#65, sum#66, count#67] - -(26) Exchange -Input [5]: [count#63, sum#64, count#65, sum#66, count#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] - -(27) HashAggregate [codegen id : 2] -Input [5]: [count#63, sum#64, count#65, sum#66, count#67] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#55)), avg(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [3]: [count(1)#68, avg(UnscaledValue(ss_ext_discount_amt#55))#69, avg(UnscaledValue(ss_net_paid#56))#70] -Results [3]: [count(1)#68 AS count(1)#71, cast((avg(UnscaledValue(ss_ext_discount_amt#55))#69 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#72, cast((avg(UnscaledValue(ss_net_paid#56))#70 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#73] +Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -(28) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#71, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#72, avg(ss_net_paid), avg(ss_net_paid)#73) AS mergedValue#74] -Input [3]: [count(1)#71, avg(ss_ext_discount_amt)#72, avg(ss_net_paid)#73] +Subquery:7 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#6, [id=#3] +Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#6, [id=#3] +Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#4] -* Project (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet spark_catalog.default.store_sales (29) +Subquery:10 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] +Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -(29) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#75, ss_ext_discount_amt#76, ss_net_paid#77, ss_sold_date_sk#78] -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 - -(30) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#75, ss_ext_discount_amt#76, ss_net_paid#77, ss_sold_date_sk#78] - -(31) Filter [codegen id : 1] -Input [4]: [ss_quantity#75, ss_ext_discount_amt#76, ss_net_paid#77, ss_sold_date_sk#78] -Condition : ((isnotnull(ss_quantity#75) AND (ss_quantity#75 >= 61)) AND (ss_quantity#75 <= 80)) - -(32) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#76, ss_net_paid#77] -Input [4]: [ss_quantity#75, ss_ext_discount_amt#76, ss_net_paid#77, ss_sold_date_sk#78] - -(33) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#76, ss_net_paid#77] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#76)), partial_avg(UnscaledValue(ss_net_paid#77))] -Aggregate Attributes [5]: [count#79, sum#80, count#81, sum#82, count#83] -Results [5]: [count#84, sum#85, count#86, sum#87, count#88] - -(34) Exchange -Input [5]: [count#84, sum#85, count#86, sum#87, count#88] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] - -(35) HashAggregate [codegen id : 2] -Input [5]: [count#84, sum#85, count#86, sum#87, count#88] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#76)), avg(UnscaledValue(ss_net_paid#77))] -Aggregate Attributes [3]: [count(1)#89, avg(UnscaledValue(ss_ext_discount_amt#76))#90, avg(UnscaledValue(ss_net_paid#77))#91] -Results [3]: [count(1)#89 AS count(1)#92, cast((avg(UnscaledValue(ss_ext_discount_amt#76))#90 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#93, cast((avg(UnscaledValue(ss_net_paid#77))#91 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#94] - -(36) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#92, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#93, avg(ss_net_paid), avg(ss_net_paid)#94) AS mergedValue#95] -Input [3]: [count(1)#92, avg(ss_ext_discount_amt)#93, avg(ss_net_paid)#94] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#4] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#4] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#10, [id=#5] -* Project (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet spark_catalog.default.store_sales (37) - - -(37) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#96, ss_ext_discount_amt#97, ss_net_paid#98, ss_sold_date_sk#99] -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 - -(38) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#96, ss_ext_discount_amt#97, ss_net_paid#98, ss_sold_date_sk#99] - -(39) Filter [codegen id : 1] -Input [4]: [ss_quantity#96, ss_ext_discount_amt#97, ss_net_paid#98, ss_sold_date_sk#99] -Condition : ((isnotnull(ss_quantity#96) AND (ss_quantity#96 >= 81)) AND (ss_quantity#96 <= 100)) - -(40) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#97, ss_net_paid#98] -Input [4]: [ss_quantity#96, ss_ext_discount_amt#97, ss_net_paid#98, ss_sold_date_sk#99] - -(41) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#97, ss_net_paid#98] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#97)), partial_avg(UnscaledValue(ss_net_paid#98))] -Aggregate Attributes [5]: [count#100, sum#101, count#102, sum#103, count#104] -Results [5]: [count#105, sum#106, count#107, sum#108, count#109] - -(42) Exchange -Input [5]: [count#105, sum#106, count#107, sum#108, count#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] - -(43) HashAggregate [codegen id : 2] -Input [5]: [count#105, sum#106, count#107, sum#108, count#109] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#97)), avg(UnscaledValue(ss_net_paid#98))] -Aggregate Attributes [3]: [count(1)#110, avg(UnscaledValue(ss_ext_discount_amt#97))#111, avg(UnscaledValue(ss_net_paid#98))#112] -Results [3]: [count(1)#110 AS count(1)#113, cast((avg(UnscaledValue(ss_ext_discount_amt#97))#111 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#114, cast((avg(UnscaledValue(ss_net_paid#98))#112 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#115] +Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -(44) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#113, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#114, avg(ss_net_paid), avg(ss_net_paid)#115) AS mergedValue#116] -Input [3]: [count(1)#113, avg(ss_ext_discount_amt)#114, avg(ss_net_paid)#115] +Subquery:13 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#5] +Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#5] +Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#1] 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 817fb0007f343..64692b9fa1cb3 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,79 +2,31 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (2) - 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] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),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(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,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] + HashAggregate [propagatedFilter_1,ss_ext_discount_amt,ss_net_paid,propagatedFilter_2,propagatedFilter_3,propagatedFilter_4,propagatedFilter_5] [count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid,ss_quantity] Filter [ss_quantity] ColumnarToRow InputAdapter Scan parquet spark_catalog.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) - 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 spark_catalog.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) - 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 spark_catalog.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) - 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 spark_catalog.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) - 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 spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - ReusedSubquery [mergedValue] #5 - ReusedSubquery [mergedValue] #5 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 Filter [r_reason_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanMergeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanMergeSuite.scala index b7557b42702e8..0a575f3d038e2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanMergeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PlanMergeSuite.scala @@ -339,4 +339,60 @@ class PlanMergeSuite extends QueryTest Row(8, 6)) } } + + test("SPARK-40193: Merge non-grouping scalar subqueries with different filter conditions") { + Seq(false, true).foreach { enableAQE => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString) { + val df = sql( + """ + |SELECT + | (SELECT sum(key) FROM testData WHERE key > 50), + | (SELECT sum(key) FROM testData WHERE key <= 50) + """.stripMargin) + + checkAnswer(df, Row(3775, 1275) :: 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 == 1, + "Missing or unexpected ReusedSubqueryExec in the plan") + } + } + } + + test("SPARK-40193: Merge non-grouping scalar subqueries where only one has a filter") { + Seq(false, true).foreach { enableAQE => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString, + // ObjectSerializerPruning produces different scan shapes depending on whether a Filter is + // present. Disabling the rule makes both scans identical so PlanMerger can merge them. + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> + "org.apache.spark.sql.catalyst.optimizer.ObjectSerializerPruning") { + val df = sql( + """ + |SELECT + | (SELECT sum(key) FROM testData), + | (SELECT sum(key) FROM testData WHERE key > 50) + """.stripMargin) + + checkAnswer(df, Row(5050, 3775) :: 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 == 1, + "Missing or unexpected ReusedSubqueryExec in the plan") + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index e4b5e10f7c3be..6cd49948630da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -80,6 +80,7 @@ trait PlanStabilitySuite extends DisableAdaptiveExecutionSuite { // Do not match `id=#123` like ids as those are actually plan ids in `SubqueryExec` nodes. private val exprIdRegexp = "(?(?(plan_id=|id=#))\\d+".r + private val propagatedFilterIdRegex = "(?propagatedFilter_)\\d+".r private val clsName = this.getClass.getCanonicalName @@ -226,7 +227,11 @@ trait PlanStabilitySuite extends DisableAdaptiveExecutionSuite { s"$padding$thisNode\n${subqueriesSimplified.mkString("")}${childrenSimplified.mkString("")}" } - simplifyNode(plan, 0) + val simplified = simplifyNode(plan, 0) + val propagatedFilterIdMap = new mutable.HashMap[String, String]() + propagatedFilterIdRegex.replaceAllIn(simplified, + m => propagatedFilterIdMap.getOrElseUpdate( + s"$m", s"${m.group("prefix")}${propagatedFilterIdMap.size + 1}")) } private def normalizeIds(plan: String): String = { @@ -237,8 +242,14 @@ trait PlanStabilitySuite extends DisableAdaptiveExecutionSuite { // Normalize the plan ids in Exchange and Subquery nodes. // See `Exchange.stringArgs` and `SubqueryExec.stringArgs` val planIdMap = new mutable.HashMap[String, String]() - planIdRegex.replaceAllIn(exprIdNormalized, + val planIdNormalized = planIdRegex.replaceAllIn(exprIdNormalized, m => planIdMap.getOrElseUpdate(s"$m", s"${m.group("prefix")}${planIdMap.size + 1}")) + + // Normalize propagatedFilter aliases introduced by PlanMerger's filter propagation. + val propagatedFilterIdMap = new mutable.HashMap[String, String]() + propagatedFilterIdRegex.replaceAllIn(planIdNormalized, + m => propagatedFilterIdMap.getOrElseUpdate( + s"$m", s"${m.group("prefix")}${propagatedFilterIdMap.size + 1}")) } private def normalizeLocation(plan: String): String = {