diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala index 2d1e71a63a8ce..042a12eb8764e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala @@ -143,7 +143,8 @@ object MergeScalarSubqueries extends Rule[LogicalPlan] { cache(i) = cache(i).copy(plan = if (header.merged) { CTERelationDef( - createProject(header.attributes, removeReferences(header.plan, cache)), + createProject(header.attributes, + removeReferences(removePropagatedFilters(header.plan), cache)), underSubquery = true) } else { removeReferences(header.plan, cache) @@ -189,9 +190,9 @@ object MergeScalarSubqueries extends Rule[LogicalPlan] { val mappedOutput = mapAttributes(output, outputMap) val headerIndex = header.attributes.indexWhere(_.exprId == mappedOutput.exprId) subqueryIndex -> headerIndex - }.orElse{ - tryMergePlans(plan, header.plan).map { - case (mergedPlan, outputMap) => + }.orElse { + tryMergePlans(plan, header.plan, false).collect { + case (mergedPlan, outputMap, None, None, _) => val mappedOutput = mapAttributes(output, outputMap) var headerIndex = header.attributes.indexWhere(_.exprId == mappedOutput.exprId) val newHeaderAttributes = if (headerIndex == -1) { @@ -223,87 +224,383 @@ object MergeScalarSubqueries extends Rule[LogicalPlan] { } } - // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged - // plan with the attribute mapping from the new to the merged version. - // Please note that merging arbitrary plans can be complicated, the current version supports only - // some of the most important nodes. + /** + * Recursively traverse down and try merging 2 plans. + * + * Please note that merging arbitrary plans can be complicated, the current version supports only + * some of the most important nodes. + * + * @param newPlan a new plan that we want to merge to an already processed plan + * @param cachedPlan a plan that we already processed, it can be either an original plan or a + * merged version of 2 or more plans + * @param filterPropagationSupported a boolean flag that we propagate down to signal we have seen + * an `Aggregate` node where propagated filters can be merged + * @return A tuple of: + * - the merged plan, + * - the attribute mapping from the new to the merged version, + * - the 2 optional filters of both plans that we need to propagate up and merge in + * an ancestor `Aggregate` node if possible, + * - the optional accumulated extra cost of merge that we need to propagate up and + * check in the ancestor `Aggregate` node. + * The cost is optional to signal if the cost needs to be taken into account up in the + * `Aggregate` node to decide about merge. + */ private def tryMergePlans( newPlan: LogicalPlan, - cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = { - checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse( + cachedPlan: LogicalPlan, + filterPropagationSupported: Boolean): + Option[(LogicalPlan, AttributeMap[Attribute], Option[Expression], Option[Expression], + Option[Double])] = { + checkIdenticalPlans(newPlan, cachedPlan).map { outputMap => + // Currently the cost is always propagated up when `filterPropagationSupported` is true but + // later we can address cases when we don't need to take cost into account. Please find the + // details at the `Filter` node handling. + val mergeCost = if (filterPropagationSupported) Some(0d) else None + + (cachedPlan, outputMap, None, None, mergeCost) + }.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 (mergedChild, outputMap, newChildFilter, mergedChildFilter, childMergeCost) => + val (mergedProjectList, newOutputMap, newPlanFilter, mergedPlanFilter, mergeCost) = + mergeNamedExpressions(np.projectList, outputMap, cp.projectList, newChildFilter, + mergedChildFilter, childMergeCost) + val mergedPlan = Project(mergedProjectList, mergedChild) + (mergedPlan, newOutputMap, newPlanFilter, mergedPlanFilter, mergeCost) } 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 (mergedChild, outputMap, newChildFilter, mergedChildFilter, childMergeCost) => + val (mergedProjectList, newOutputMap, newPlanFilter, mergedPlanFilter, mergeCost) = + mergeNamedExpressions(np.output, outputMap, cp.projectList, newChildFilter, + mergedChildFilter, childMergeCost) + val mergedPlan = Project(mergedProjectList, mergedChild) + (mergedPlan, newOutputMap, newPlanFilter, mergedPlanFilter, mergeCost) } 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 (mergedChild, outputMap, newChildFilter, mergedChildFilter, childMergeCost) => + val (mergedProjectList, newOutputMap, newPlanFilter, mergedPlanFilter, mergeCost) = + mergeNamedExpressions(np.projectList, outputMap, cp.output, newChildFilter, + mergedChildFilter, childMergeCost) + val mergedPlan = Project(mergedProjectList, mergedChild) + (mergedPlan, newOutputMap, newPlanFilter, mergedPlanFilter, mergeCost) } 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 - } - } + val filterPropagationSupported = + conf.getConf(SQLConf.PLAN_MERGE_FILTER_PROPAGATION_ENABLED) && + supportsFilterPropagation(np) && supportsFilterPropagation(cp) + tryMergePlans(np.child, cp.child, filterPropagationSupported).flatMap { + case (mergedChild, outputMap, None, None, _) => + 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)) { + // No need to calculate and check costs as there is no propagated filter + val (mergedAggregateExpressions, newOutputMap, _, _, _) = + mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions, + None, None, None) + val mergedPlan = + Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild) + Some(mergedPlan, newOutputMap, None, None, None) + } else { + None + } + case (mergedChild, outputMap, newChildFilter, mergedChildFilter, childMergeCost) => + // No need to calculate cost in `mergeNamedExpressions()` + val (mergedAggregateExpressions, newOutputMap, _, _, _) = + mergeNamedExpressions( + filterAggregateExpressions(np.aggregateExpressions, newChildFilter), + outputMap, + filterAggregateExpressions(cp.aggregateExpressions, mergedChildFilter), + None, + None, + None) - case (np: Filter, cp: Filter) => - tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) => - val mappedNewCondition = mapAttributes(np.condition, outputMap) - // Comparing the canonicalized form is required to ignore different forms of the same - // expression. - if (mappedNewCondition.canonicalized == cp.condition.canonicalized) { - val mergedPlan = cp.withNewChildren(Seq(mergedChild)) - Some(mergedPlan -> outputMap) - } else { - None - } + val mergeFilters = newChildFilter.isEmpty || mergedChildFilter.isEmpty || { + val mergeCost = childMergeCost.map { c => + val newPlanExtraCost = mergedChildFilter.map(getCost).getOrElse(0d) + + newChildFilter.map(getCost).getOrElse(0d) + val cachedPlanExtraCost = newPlanExtraCost + c + newPlanExtraCost + cachedPlanExtraCost + } + mergeCost.forall { c => + val maxCost = conf.getConf(SQLConf.PLAN_MERGE_FILTER_PROPAGATION_MAX_COST) + val enableMerge = maxCost < 0 || c <= maxCost + if (!enableMerge) { + logDebug( + s"Plan merge of\n${np}and\n${cp}failed as the merge cost is too high: $c") + } + enableMerge + } + } + if (mergeFilters) { + val mergedPlan = Aggregate(Seq.empty, mergedAggregateExpressions, mergedChild) + Some(mergedPlan, newOutputMap, None, None, None) + } else { + None + } + case _ => None } - case (np: Join, cp: Join) if np.joinType == cp.joinType && np.hint == cp.hint => - tryMergePlans(np.left, cp.left).flatMap { case (mergedLeft, leftOutputMap) => - tryMergePlans(np.right, cp.right).flatMap { case (mergedRight, rightOutputMap) => - val outputMap = leftOutputMap ++ rightOutputMap - val mappedNewCondition = np.condition.map(mapAttributes(_, outputMap)) + // If `Filter` conditions are not exactly the same we can still try propagating up their + // differing conditions because in some cases we will be able to merge them in an + // aggregate parent node. + // + // The differing `Filter`s can be merged if: + // - they both they have an ancestor `Aggregate` node that has no grouping and + // - there are only `Project` or `Filter` nodes in between the differing `Filter` and the + // ancestor `Aggregate` nodes. + // + // E.g. we can merge: + // + // SELECT avg(a) FROM t WHERE c = 1 + // + // and: + // + // SELECT sum(b) FROM t WHERE c = 2 + // + // into: + // + // SELECT + // avg(a) FILTER (WHERE c = 1), + // sum(b) FILTER (WHERE c = 2) + // FORM t + // WHERE c = 1 OR c = 2 + // + // But there are some special cases we need to consider: + // + // - The plans to be merged might contain multiple adjacent `Filter` nodes and the parent + // `Filter` nodes should incorporate the propagated filters from child ones during merge. + // + // E.g. adjacent filters can appear in plans when some of the optimization rules (like + // `PushDownPredicates`) are disabled. + // + // Let's consider we want to merge query 1: + // + // SELECT avg(a) + // FROM ( + // SELECT * FROM t WHERE c1 = 1 + // ) + // WHERE c2 = 1 + // + // and query 2: + // + // SELECT sum(b) + // FROM ( + // SELECT * FROM t WHERE c1 = 2 + // ) + // WHERE c2 = 2 + // + // then the optimal merged query is: + // + // SELECT + // avg(a) FILTER (WHERE c1 = 1 AND c2 = 1), + // sum(b) FILTER (WHERE c1 = 2 AND c2 = 2) + // FORM ( + // SELECT * FROM t WHERE c1 = 1 OR c1 = 2 + // ) + // WHERE (c1 = 1 AND c2 = 1) OR (c1 = 2 AND c2 = 2) + // + // This is because the `WHERE (c1 = 1 AND c2 = 1) OR (c1 = 2 AND c2 = 2)` parent `Filter` + // condition is more selective than a simple `WHERE c2 = 1 OR c2 = 2` would be as the + // simple condition would let trough rows containing c1 = 1 and c2 = 2, which none of the + // original queries do. + // + // - When we merge plans to already merged plans the propagated filter conditions could grow + // quickly, which we can avoid with tagging the already propagated filters. + // + // E.g. if we merged the previous optimal merged query and query 3: + // + // SELECT max(b) + // FROM ( + // SELECT * FROM t WHERE c1 = 3 + // ) + // WHERE c2 = 3 + // + // then a new double-merged query would look like this: + // + // SELECT + // avg(a) FILTER (WHERE + // (c1 = 1 AND c2 = 1) AND + // ((c1 = 1 OR c1 = 2) AND ((c1 = 1 AND c2 = 1) OR (c1 = 2 AND c2 = 2))) + // ), + // sum(b) FILTER (WHERE + // (c1 = 2 AND c2 = 2) AND + // ((c1 = 1 OR c1 = 2) AND ((c1 = 1 AND c2 = 1) OR (c1 = 2 AND c2 = 2))) + // ), + // max(b) FILTER (WHERE c1 = 3 AND c2 = 3) + // FORM ( + // SELECT * FROM t WHERE (c1 = 1 OR c1 = 2) OR c1 = 3 + // ) + // WHERE + // ((c1 = 1 OR c1 = 2) AND ((c1 = 1 AND c2 = 1) OR (c1 = 2 AND c2 = 2))) OR + // (c1 = 3 AND c2 = 3) + // + // which is not optimal and contains unnecessary complex conditions. + // + // Please note that `BooleanSimplification` and other rules could help simplifying filter + // conditions, but when we merge large number if queries in this rule, the plan size can + // increase exponentially and can cause memory issues before `BooleanSimplification` could + // run. + // + // We can avoid that complexity if we tag already propagated filter conditions with a + // simple `PropagatedFilter` wrapper during merge. + // E.g. the actual merged query of query 1 and query 2 produced by this rule looks like + // this: + // + // SELECT + // avg(a) FILTER (WHERE c1 = 1 AND c2 = 1), + // sum(b) FILTER (WHERE c1 = 2 AND c2 = 2) + // FORM ( + // SELECT * FROM t WHERE PropagatedFilter(c1 = 1 OR c1 = 2) + // ) + // WHERE PropagatedFilter((c1 = 1 AND c2 = 1) OR (c1 = 2 AND c2 = 2)) + // + // And so when we merge query 3 we know that filter conditions tagged with + // `PropagatedFilter` can be ignored during filter propagation and thus the we get a much + // simpler double-merged query: + // + // SELECT + // avg(a) FILTER (WHERE c1 = 1 AND c2 = 1), + // sum(b) FILTER (WHERE c1 = 2 AND c2 = 2), + // max(b) FILTER (WHERE c1 = 3 AND c2 = 3) + // FORM ( + // SELECT * FROM t WHERE PropagatedFilter(PropagatedFilter(c1 = 1 OR c1 = 2) OR c1 = 3) + // ) + // WHERE + // PropagatedFilter( + // PropagatedFilter((c1 = 1 AND c2 = 1) OR (c1 = 2 AND c2 = 2)) OR + // (c1 = 3 AND c2 = 3) + // + // At the end of the rule we remove the `PropagatedFilter` wrappers. + // + // - When we merge plans we might introduce performance degradation in some corner cases. + // The performance improvement of a merged query are due to: + // - Spark needs scan the underlying common data source only once, + // - common data needs to be shuffled only once between partial and final aggregates, + // - there can be common expressions in the original plans that needs to be executed only + // once. + // But despite the above advantages, when differing filters can be pushed down to data + // sources in the physical plan the original data source scans might not overlap and + // return disjoint set of rows and so cancel out the above gains of merging. In this + // unfortunate corner case if there are no common expressions in the original plans then + // the merged query executes all expressions of both original queries on all rows from + // both original queries. + // + // As plan merge works on logical plans, identifying the above corner cases is + // non-trivial. So to minimize the possible performance degradation we allow merging only + // if: + // - any of the propagated filters to the ancestor `Aggregate` node is empty, which means + // that scans surely overlap, or + // - the sum of cost differences between the original plans and the merged plan is low. + // + // Currently we measure the cost of plans with a very simple `getCost` function that + // allows only the most basic expressions. + case (np: Filter, cp: Filter) => + tryMergePlans(np.child, cp.child, filterPropagationSupported).flatMap { + case (mergedChild, outputMap, newChildFilter, mergedChildFilter, childMergeCost) => + val mappedNewCondition = mapAttributes(np.condition, outputMap) // Comparing the canonicalized form is required to ignore different forms of the same - // expression and `AttributeReference.quailifier`s in `cp.condition`. - if (mappedNewCondition.map(_.canonicalized) == cp.condition.map(_.canonicalized)) { - val mergedPlan = cp.withNewChildren(Seq(mergedLeft, mergedRight)) - Some(mergedPlan -> outputMap) + // expression. + if (mappedNewCondition.canonicalized == cp.condition.canonicalized) { + val filters = (mergedChildFilter.toSeq ++ newChildFilter.toSeq).reduceOption(Or) + .map(PropagatedFilter) + // Please note that: + // - here we construct the merged `Filter` condition in a way that the filters we + // propagate are wrapped by `PropagatedFilter` and are on the left side of the + // `And` condition + // - at other places we always construct the merge condition that it is fully + // wrapped by `PropagatedFilter` + // so as to be able to extract the already propagated filters in + // `extractNonPropagatedFilter()` easily. + val mergedCondition = (filters.toSeq :+ cp.condition).reduce(And) + val mergedPlan = Filter(mergedCondition, mergedChild) + val mergeCost = addFilterCost(childMergeCost, mergedCondition, + getCost(np.condition), getCost(cp.condition)) + Some(mergedPlan, outputMap, newChildFilter, mergedChildFilter, mergeCost) + } else if (filterPropagationSupported) { + val newPlanFilter = (newChildFilter.toSeq :+ mappedNewCondition).reduce(And) + val cachedPlanFilter = (mergedChildFilter.toSeq :+ cp.condition).reduce(And) + val mergedCondition = PropagatedFilter(Or(cachedPlanFilter, newPlanFilter)) + val mergedPlan = Filter(mergedCondition, mergedChild) + // There might be `PropagatedFilter`s in the cached plan's `Filter` that we don't + // need to re-propagate. + val nonPropagatedCachedFilter = extractNonPropagatedFilter(cp.condition) + val mergedPlanFilter = + (mergedChildFilter.toSeq ++ nonPropagatedCachedFilter.toSeq).reduceOption(And) + val mergeCost = addFilterCost(childMergeCost, mergedCondition, + getCost(np.condition), getCost(cp.condition)) + Some(mergedPlan, outputMap, Some(newPlanFilter), mergedPlanFilter, mergeCost) } else { None } - } + } + case (np, cp: Filter) if filterPropagationSupported => + tryMergePlans(np, cp.child, true).map { + case (mergedChild, outputMap, newChildFilter, mergedChildFilter, childMergeCost) => + // There might be `PropagatedFilter`s in the cached plan's `Filter` and we don't + // need to re-propagate them. + val nonPropagatedCachedFilter = extractNonPropagatedFilter(cp.condition) + val mergedPlanFilter = + (mergedChildFilter.toSeq ++ nonPropagatedCachedFilter.toSeq).reduceOption(And) + if (newChildFilter.isEmpty) { + (mergedChild, outputMap, None, mergedPlanFilter, childMergeCost) + } else { + val cachedPlanFilter = (mergedChildFilter.toSeq :+ cp.condition).reduce(And) + val mergedCondition = PropagatedFilter(Or(cachedPlanFilter, newChildFilter.get)) + val mergedPlan = Filter(mergedCondition, mergedChild) + val mergeCost = + addFilterCost(childMergeCost, mergedCondition, 0d, getCost(cp.condition)) + (mergedPlan, outputMap, newChildFilter, mergedPlanFilter, mergeCost) + } + } + case (np: Filter, cp) if filterPropagationSupported => + tryMergePlans(np.child, cp, true).map { + case (mergedChild, outputMap, newChildFilter, mergedChildFilter, childMergeCost) => + val mappedNewCondition = mapAttributes(np.condition, outputMap) + val newPlanFilter = (newChildFilter.toSeq :+ mappedNewCondition).reduce(And) + if (mergedChildFilter.isEmpty) { + (mergedChild, outputMap, Some(newPlanFilter), None, childMergeCost) + } else { + val mergedCondition = PropagatedFilter(Or(mergedChildFilter.get, newPlanFilter)) + val mergedPlan = Filter(mergedCondition, mergedChild) + val mergeCost = + addFilterCost(childMergeCost, mergedCondition, getCost(np.condition), 0d) + (mergedPlan, outputMap, Some(newPlanFilter), mergedChildFilter, mergeCost) + } + } + + case (np: Join, cp: Join) if np.joinType == cp.joinType && np.hint == cp.hint => + // Filter propagation is not allowed through joins + tryMergePlans(np.left, cp.left, false).flatMap { + case (mergedLeft, leftOutputMap, None, None, _) => + tryMergePlans(np.right, cp.right, false).flatMap { + case (mergedRight, rightOutputMap, None, None, _) => + val outputMap = leftOutputMap ++ rightOutputMap + val mappedNewCondition = np.condition.map(mapAttributes(_, outputMap)) + // Comparing the canonicalized form is required to ignore different forms of the + // same expression and `AttributeReference.quailifier`s in `cp.condition`. + if (mappedNewCondition.map(_.canonicalized) == + cp.condition.map(_.canonicalized)) { + val mergedPlan = cp.withNewChildren(Seq(mergedLeft, mergedRight)) + Some(mergedPlan, outputMap, None, None, None) + } else { + None + } + case _ => None + } + case _ => None } // Otherwise merging is not possible. case _ => None - }) + } + ) } private def createProject(attributes: Seq[Attribute], plan: LogicalPlan): Project = { @@ -320,14 +617,43 @@ object MergeScalarSubqueries extends Rule[LogicalPlan] { }.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. + /** + * Merges named expression lists of `Project` or `Aggregate` nodes of the new plan into the named + * expression list of a similar node of the cached plan. + * + * - Before we can merge the new expressions we need to take into account the propagated + * attribute mapping that describes the transformation from the input attributes of the new plan + * node to the output attributes of the already merged child plan node. + * - While merging the new expressions we need to build a new attribute mapping to propagate up. + * - If any filters are propagated from `Filter` nodes below then we could add all the referenced + * attributes of filter conditions to the merged expression list, but it is better if we alias + * whole filter conditions and propagate only the new boolean attributes. + * + * @param newExpressions the expression list of the new plan node + * @param outputMap the propagated attribute mapping + * @param cachedExpressions the expression list of the cached plan node + * @param newChildFilter the propagated filters from `Filter` nodes of the new plan + * @param mergedChildFilter the propagated filters from `Filter` nodes of the merged child plan + * @param childMergeCost the optional accumulated extra costs of merge + * @return A tuple of: + * - the merged expression list, + * - the new attribute mapping to propagate, + * - the output attribute of the merged newChildFilter to propagate, + * - the output attribute of the merged mergedChildFilter to propagate, + * - the extra costs of merging new expressions and filters added to `childMergeCost` + */ private def mergeNamedExpressions( newExpressions: Seq[NamedExpression], outputMap: AttributeMap[Attribute], - cachedExpressions: Seq[NamedExpression]) = { + cachedExpressions: Seq[NamedExpression], + newChildFilter: Option[Expression], + mergedChildFilter: Option[Expression], + childMergeCost: Option[Double]): + (Seq[NamedExpression], AttributeMap[Attribute], Option[Attribute], Option[Attribute], + Option[Double]) = { val mergedExpressions = ArrayBuffer[NamedExpression](cachedExpressions: _*) + val commonCachedExpressions = mutable.Set.empty[NamedExpression] + var cachedPlanExtraCost = 0d val newOutputMap = AttributeMap(newExpressions.map { ne => val mapped = mapAttributes(ne, outputMap) val withoutAlias = mapped match { @@ -337,12 +663,82 @@ object MergeScalarSubqueries extends Rule[LogicalPlan] { ne.toAttribute -> mergedExpressions.find { case Alias(child, _) => child semanticEquals withoutAlias case e => e semanticEquals withoutAlias + }.map { e => + if (childMergeCost.isDefined) { + commonCachedExpressions += e + } + e }.getOrElse { mergedExpressions += mapped + if (childMergeCost.isDefined) { + cachedPlanExtraCost += getCost(mapped) + } mapped }.toAttribute }) - (mergedExpressions.toSeq, newOutputMap) + + def mergeFilter(filter: Option[Expression]) = { + filter.map { f => + mergedExpressions.find { + case Alias(child, _) => child semanticEquals f + case e => e semanticEquals f + }.map { e => + if (childMergeCost.isDefined) { + commonCachedExpressions += e + } + e + }.getOrElse { + val named = f match { + case ne: NamedExpression => ne + case o => Alias(o, "propagatedFilter")() + } + mergedExpressions += named + if (childMergeCost.isDefined) { + cachedPlanExtraCost += getCost(named) + } + named + }.toAttribute + } + } + + val mergedPlanFilter = mergeFilter(mergedChildFilter) + val newPlanFilter = mergeFilter(newChildFilter) + + val mergeCost = childMergeCost.map { c => + val newPlanExtraCost = cachedExpressions.collect { + case e if !commonCachedExpressions.contains(e) => getCost(e) + }.sum + c + newPlanExtraCost + cachedPlanExtraCost + } + + (mergedExpressions.toSeq, newOutputMap, newPlanFilter, mergedPlanFilter, mergeCost) + } + + /** + * Adds the extra cost of using `mergedCondition` (instead of the original cost of new and cached + * plan filter conditions) to the propagated extra cost from merged child plans. + */ + private def addFilterCost( + childMergeCost: Option[Double], + mergedCondition: Expression, + newPlanFilterCost: Double, + cachedPlanFilterCost: Double) = { + childMergeCost.map { c => + val mergedConditionCost = getCost(mergedCondition) + val newPlanExtraCost = mergedConditionCost - newPlanFilterCost + val cachedPlanExtraCost = mergedConditionCost - cachedPlanFilterCost + c + newPlanExtraCost + cachedPlanExtraCost + } + } + + // Currently only the most basic expressions are supported. + private def getCost(e: Expression): Double = e match { + case _: Literal | _: Attribute => 0d + case PropagatedFilter(child) => getCost(child) + case Alias(child, _) => getCost(child) + case _: BinaryComparison | _: BinaryArithmetic | _: And | _: Or | _: IsNull | _: IsNotNull => + 1d + e.children.map(getCost).sum + case _ => Double.PositiveInfinity } // Only allow aggregates of the same implementation because merging different implementations @@ -373,6 +769,38 @@ object MergeScalarSubqueries extends Rule[LogicalPlan] { } } + private def extractNonPropagatedFilter(e: Expression) = { + e match { + case And(_: PropagatedFilter, e) => Some(e) + case _: PropagatedFilter => None + case o => Some(o) + } + } + + // We allow filter propagation into aggregates that doesn't have grouping expressions. + private def supportsFilterPropagation(a: Aggregate) = { + a.groupingExpressions.isEmpty + } + + private def filterAggregateExpressions( + aggregateExpressions: Seq[NamedExpression], + filter: Option[Expression]) = { + if (filter.isDefined) { + aggregateExpressions.map(_.transform { + case ae: AggregateExpression => + ae.copy(filter = (filter.get +: ae.filter.toSeq).reduceOption(And)) + }.asInstanceOf[NamedExpression]) + } else { + aggregateExpressions + } + } + + private def removePropagatedFilters(plan: LogicalPlan) = { + plan.transformAllExpressions { + case pf: PropagatedFilter => pf.child + } + } + // Second traversal replaces `ScalarSubqueryReference`s to either // `GetStructField(ScalarSubquery(CTERelationRef to the merged plan)` if the plan is merged from // multiple subqueries or `ScalarSubquery(original plan)` if it isn't. @@ -418,3 +846,14 @@ case class ScalarSubqueryReference( override def stringArgs: Iterator[Any] = Iterator(subqueryIndex, headerIndex, dataType, exprId.id) } + + +/** + * Temporal wrapper around already propagated predicates. + */ +case class PropagatedFilter(child: Expression) extends UnaryExpression with Unevaluable { + override def dataType: DataType = child.dataType + + override protected def withNewChildInternal(newChild: Expression): PropagatedFilter = + copy(child = newChild) +} 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 3b6374e712c66..669a3096b4d38 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 @@ -4815,6 +4815,24 @@ object SQLConf { .booleanConf .createOptional + val PLAN_MERGE_FILTER_PROPAGATION_ENABLED = + buildConf("spark.sql.planMerge.filterPropagation.enabled") + .internal() + .doc(s"When set to true different filters can be propagated up to aggregates.") + .version("4.0.0") + .booleanConf + .createWithDefault(true) + + val PLAN_MERGE_FILTER_PROPAGATION_MAX_COST = + buildConf("spark.sql.planMerge.filterPropagation.maxCost") + .internal() + .doc("The maximum allowed additional cost of merging. By setting this value to -1 filter " + + "propagation is always allowed.") + .version("4.0.0") + .doubleConf + .checkValue(c => c >= 0 || c == -1, "The maximum allowed cost must not be negative") + .createWithDefault(100) + 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. The MINIMAL and STANDARD formats are pretty JSON formats where " + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala index b3444b0b43077..23c56c75680c2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, Collect 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 MergeScalarSubqueriesSuite extends PlanTest { @@ -611,4 +612,279 @@ class MergeScalarSubqueriesSuite extends PlanTest { comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } + + test("Merging subqueries with different filters") { + Seq(true, false).foreach { mergeEnabled => + withSQLConf(SQLConf.PLAN_MERGE_FILTER_PROPAGATION_ENABLED.key -> s"$mergeEnabled") { + val subquery1 = + ScalarSubquery(testRelation.where($"b" > 0).groupBy()(max($"a").as("max_a"))) + val subquery2 = + ScalarSubquery(testRelation.where($"b" < 0).groupBy()(sum($"a").as("sum_a"))) + val subquery3 = + ScalarSubquery(testRelation.where($"b" === 0).groupBy()(avg($"a").as("avg_a"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2, + subquery3) + + val correctAnswer = if (mergeEnabled) { + val mergedSubquery = testRelation + .where($"b" > 0 || $"b" < 0 || $"b" === 0) + .groupBy()( + max($"a", Some($"b" > 0)).as("max_a"), + sum($"a", Some($"b" < 0)).as("sum_a"), + avg($"a", Some($"b" === 0)).as("avg_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("sum_a"), $"sum_a", + Literal("avg_a"), $"avg_a" + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1), + extractorExpression(0, analyzedMergedSubquery.output, 2)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + } else { + originalQuery + } + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } + } + + test("Merging subqueries with different filters - cost limit") { + val subquery1 = ScalarSubquery(testRelation.where($"b" > 0).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"b" < 0).groupBy()(sum($"a").as("sum_a"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2) + Seq(0, 7, 8, -1).foreach { maxCost => + CTERelationDef.curId.set(0) + withSQLConf(SQLConf.PLAN_MERGE_FILTER_PROPAGATION_MAX_COST.key -> s"$maxCost") { + // Extra cost of mergedSubquery to subquery1 is 4: `b < 0` and `||` in the merged + // `Filter`, and `b > 0`, `b < 0` in the merged `Aggregate` nodes. + // Extra cost of mergedSubquery to subquery2 is 4: `b > 0` and `||` in the merged + // `Filter`, and `b > 0`, `b < 0` in the merged `Aggregate` nodes. + val correctAnswer = if (maxCost < 0 || maxCost >= 8) { + val mergedSubquery = testRelation + .where($"b" > 0 || $"b" < 0) + .groupBy()( + max($"a", Some($"b" > 0)).as("max_a"), + sum($"a", Some($"b" < 0)).as("sum_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("sum_a"), $"sum_a" + )).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 1)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + } else { + originalQuery + } + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } + } + + test("Merging subqueries with and without filters") { + val subquery1 = ScalarSubquery(testRelation.where($"b" > 0).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.groupBy()(count($"a").as("cnt_a"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2) + val mergedSubquery = testRelation + .groupBy()( + max($"a", Some($"b" > 0)).as("max_a"), + count($"a").as("cnt_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("cnt_a"), $"cnt_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))) + + // If one side doesn't contain a filter then physical scans surely overlap so merging is cost + // independent. + Seq(0, -1).foreach { maxCost => + CTERelationDef.curId.set(0) + withSQLConf(SQLConf.PLAN_MERGE_FILTER_PROPAGATION_MAX_COST.key -> s"$maxCost") { + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } + } + + test("Merging subqueries without and with filters") { + val subquery1 = ScalarSubquery(testRelation.groupBy()(count($"a").as("cnt_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"b" > 0).groupBy()(max($"a").as("max_a"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2) + + val mergedSubquery = testRelation + .groupBy()( + count($"a").as("cnt_a"), + max($"a", Some($"b" > 0)).as("max_a")) + .select(CreateNamedStruct(Seq( + Literal("cnt_a"), $"cnt_a", + Literal("max_a"), $"max_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))) + + // If one side doesn't contain a filter then physical scans surely overlap so merging is cost + // independent. + Seq(0, -1).foreach { maxCost => + CTERelationDef.curId.set(0) + withSQLConf(SQLConf.PLAN_MERGE_FILTER_PROPAGATION_MAX_COST.key -> s"$maxCost") { + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } + } + + test("Merging subqueries with same condition in filter and in having") { + val subquery1 = ScalarSubquery(testRelation.where($"b" > 0).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.groupBy()(max($"a", Some($"b" > 0)).as("max_a_2"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2) + + val mergedSubquery = testRelation + .groupBy()( + max($"a", Some($"b" > 0)).as("max_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a")).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 0)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Merging subqueries with same condition in having and in filter") { + val subquery1 = ScalarSubquery(testRelation.groupBy()(max($"a", Some($"b" > 0)).as("max_a"))) + val subquery2 = ScalarSubquery(testRelation.where($"b" > 0).groupBy()(max($"a").as("max_a_2"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2) + + val mergedSubquery = testRelation + .groupBy()( + max($"a", Some($"b" > 0)).as("max_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a")).as("mergedValue")) + val analyzedMergedSubquery = mergedSubquery.analyze + val correctAnswer = WithCTE( + testRelation + .select( + extractorExpression(0, analyzedMergedSubquery.output, 0), + extractorExpression(0, analyzedMergedSubquery.output, 0)), + Seq(definitionNode(analyzedMergedSubquery, 0))) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Merging subqueries with different filters, multiple filters propagated") { + val subquery1 = ScalarSubquery( + testRelation.where($"b" > 0).where($"c" === "a").groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery( + testRelation.where($"b" > 0).where($"c" === "b").groupBy()(avg($"a").as("avg_a"))) + val subquery3 = ScalarSubquery( + testRelation.where($"b" < 0).where($"c" === "c").groupBy()(count($"a").as("cnt_a"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2, + subquery3) + + val mergedSubquery = testRelation + .where($"b" > 0 || $"b" < 0) + .where($"b" > 0 && ($"c" === "a" || $"c" === "b") || $"b" < 0 && $"c" === "c") + .groupBy()( + max($"a", Some($"b" > 0 && $"c" === "a")).as("max_a"), + avg($"a", Some($"b" > 0 && $"c" === "b")).as("avg_a"), + count($"a", Some($"b" < 0 && $"c" === "c")).as("cnt_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("avg_a"), $"avg_a", + Literal("cnt_a"), $"cnt_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("Merging subqueries with different filters, multiple filters propagated 2") { + val subquery1 = ScalarSubquery( + testRelation.where($"c" === "a").where($"b" > 0).groupBy()(max($"a").as("max_a"))) + val subquery2 = ScalarSubquery( + testRelation.where($"c" === "b").where($"b" > 0).groupBy()(avg($"a").as("avg_a"))) + val subquery3 = ScalarSubquery( + testRelation.where($"c" === "c").where($"b" < 0).groupBy()(count($"a").as("cnt_a"))) + val originalQuery = testRelation + .select( + subquery1, + subquery2, + subquery3) + + val mergedSubquery = testRelation + .where($"c" === "a" || $"c" === "b" || $"c" === "c") + .where(($"c" === "a" || $"c" === "b") && $"b" > 0 || $"c" === "c" && $"b" < 0) + .groupBy()( + // Note: `b` related conditions are evaluated first despite `c` related ones are lower in + // the original plans. This is because `$"b" > 0` is the same in the first 2 original plans + // so it isn't propagated when we merge them. But later when we merge the 3rd plan, it is. + max($"a", Some($"b" > 0 && $"c" === "a")).as("max_a"), + avg($"a", Some($"b" > 0 && $"c" === "b")).as("avg_a"), + count($"a", Some($"c" === "c" && $"b" < 0)).as("cnt_a")) + .select(CreateNamedStruct(Seq( + Literal("max_a"), $"max_a", + Literal("avg_a"), $"avg_a", + Literal("cnt_a"), $"cnt_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) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 3382a1161ddba..d0ad9ab48605b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -63,7 +63,7 @@ class SparkOptimizer( RewriteDistinctAggregates) :+ Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates) :+ - Batch("Cleanup filters that cannot be pushed down", Once, + Batch("Cleanup filters that cannot be pushed down", FixedPoint(1), CleanupDynamicPruningFilters, // cleanup the unnecessary TrueLiteral predicates BooleanSimplification, 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 5313a1336fee4..537193bd49b2c 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=#8], 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=#8].bloomFilter, xxhash64(d_week_seq#5, 42))) (7) BroadcastExchange Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] @@ -203,7 +203,7 @@ Input [3]: [d_date_sk#55, d_week_seq#56, d_day_name#57] (31) Filter [codegen id : 5] Input [3]: [d_date_sk#55, d_week_seq#56, d_day_name#57] -Condition : ((isnotnull(d_date_sk#55) AND isnotnull(d_week_seq#56)) AND might_contain(Subquery scalar-subquery#58, [id=#59], xxhash64(d_week_seq#56, 42))) +Condition : ((isnotnull(d_date_sk#55) AND isnotnull(d_week_seq#56)) AND might_contain(ReusedSubquery Subquery scalar-subquery#7, [id=#8].bloomFilter, xxhash64(d_week_seq#56, 42))) (32) BroadcastExchange Input [3]: [d_date_sk#55, d_week_seq#56, d_day_name#57] @@ -223,115 +223,116 @@ Input [6]: [ss_store_sk#52, ss_sales_price#53, ss_sold_date_sk#54, d_date_sk#55, Input [4]: [ss_store_sk#52, ss_sales_price#53, d_week_seq#56, d_day_name#57] Keys [2]: [d_week_seq#56, ss_store_sk#52] Functions [6]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Sunday ) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Monday ) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Wednesday) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Thursday ) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Friday ) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Saturday ) THEN ss_sales_price#53 END))] -Aggregate Attributes [6]: [sum#60, sum#61, sum#62, sum#63, sum#64, sum#65] -Results [8]: [d_week_seq#56, ss_store_sk#52, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] +Aggregate Attributes [6]: [sum#58, sum#59, sum#60, sum#61, sum#62, sum#63] +Results [8]: [d_week_seq#56, ss_store_sk#52, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69] (36) Exchange -Input [8]: [d_week_seq#56, ss_store_sk#52, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] +Input [8]: [d_week_seq#56, ss_store_sk#52, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69] Arguments: hashpartitioning(d_week_seq#56, ss_store_sk#52, 5), ENSURE_REQUIREMENTS, [plan_id=6] (37) HashAggregate [codegen id : 9] -Input [8]: [d_week_seq#56, ss_store_sk#52, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] +Input [8]: [d_week_seq#56, ss_store_sk#52, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69] Keys [2]: [d_week_seq#56, ss_store_sk#52] Functions [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#57 = Sunday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Monday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Wednesday) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Thursday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Friday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Saturday ) THEN ss_sales_price#53 END))] Aggregate Attributes [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#57 = Sunday ) THEN ss_sales_price#53 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Monday ) THEN ss_sales_price#53 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Wednesday) THEN ss_sales_price#53 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Thursday ) THEN ss_sales_price#53 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Friday ) THEN ss_sales_price#53 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Saturday ) THEN ss_sales_price#53 END))#29] -Results [8]: [d_week_seq#56, ss_store_sk#52, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Sunday ) THEN ss_sales_price#53 END))#23,17,2) AS sun_sales#72, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Monday ) THEN ss_sales_price#53 END))#24,17,2) AS mon_sales#73, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Wednesday) THEN ss_sales_price#53 END))#26,17,2) AS wed_sales#74, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Thursday ) THEN ss_sales_price#53 END))#27,17,2) AS thu_sales#75, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Friday ) THEN ss_sales_price#53 END))#28,17,2) AS fri_sales#76, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Saturday ) THEN ss_sales_price#53 END))#29,17,2) AS sat_sales#77] +Results [8]: [d_week_seq#56, ss_store_sk#52, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Sunday ) THEN ss_sales_price#53 END))#23,17,2) AS sun_sales#70, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Monday ) THEN ss_sales_price#53 END))#24,17,2) AS mon_sales#71, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Wednesday) THEN ss_sales_price#53 END))#26,17,2) AS wed_sales#72, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Thursday ) THEN ss_sales_price#53 END))#27,17,2) AS thu_sales#73, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Friday ) THEN ss_sales_price#53 END))#28,17,2) AS fri_sales#74, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Saturday ) THEN ss_sales_price#53 END))#29,17,2) AS sat_sales#75] (38) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#78, s_store_id#79] +Output [2]: [s_store_sk#76, s_store_id#77] 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#78, s_store_id#79] +Input [2]: [s_store_sk#76, s_store_id#77] (40) Filter [codegen id : 7] -Input [2]: [s_store_sk#78, s_store_id#79] -Condition : (isnotnull(s_store_sk#78) AND isnotnull(s_store_id#79)) +Input [2]: [s_store_sk#76, s_store_id#77] +Condition : (isnotnull(s_store_sk#76) AND isnotnull(s_store_id#77)) (41) BroadcastExchange -Input [2]: [s_store_sk#78, s_store_id#79] +Input [2]: [s_store_sk#76, s_store_id#77] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#52] -Right keys [1]: [s_store_sk#78] +Right keys [1]: [s_store_sk#76] Join type: Inner Join condition: None (43) Project [codegen id : 9] -Output [8]: [d_week_seq#56, sun_sales#72, mon_sales#73, wed_sales#74, thu_sales#75, fri_sales#76, sat_sales#77, s_store_id#79] -Input [10]: [d_week_seq#56, ss_store_sk#52, sun_sales#72, mon_sales#73, wed_sales#74, thu_sales#75, fri_sales#76, sat_sales#77, s_store_sk#78, s_store_id#79] +Output [8]: [d_week_seq#56, 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#56, ss_store_sk#52, 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] (44) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#80, d_week_seq#81] +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 (45) ColumnarToRow [codegen id : 8] -Input [2]: [d_month_seq#80, d_week_seq#81] +Input [2]: [d_month_seq#78, d_week_seq#79] (46) Filter [codegen id : 8] -Input [2]: [d_month_seq#80, d_week_seq#81] -Condition : (((isnotnull(d_month_seq#80) AND (d_month_seq#80 >= 1197)) AND (d_month_seq#80 <= 1208)) AND isnotnull(d_week_seq#81)) +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)) (47) Project [codegen id : 8] -Output [1]: [d_week_seq#81] -Input [2]: [d_month_seq#80, d_week_seq#81] +Output [1]: [d_week_seq#79] +Input [2]: [d_month_seq#78, d_week_seq#79] (48) BroadcastExchange -Input [1]: [d_week_seq#81] +Input [1]: [d_week_seq#79] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] (49) BroadcastHashJoin [codegen id : 9] Left keys [1]: [d_week_seq#56] -Right keys [1]: [d_week_seq#81] +Right keys [1]: [d_week_seq#79] Join type: Inner Join condition: None (50) Project [codegen id : 9] -Output [8]: [d_week_seq#56 AS d_week_seq2#82, s_store_id#79 AS s_store_id2#83, sun_sales#72 AS sun_sales2#84, mon_sales#73 AS mon_sales2#85, wed_sales#74 AS wed_sales2#86, thu_sales#75 AS thu_sales2#87, fri_sales#76 AS fri_sales2#88, sat_sales#77 AS sat_sales2#89] -Input [9]: [d_week_seq#56, sun_sales#72, mon_sales#73, wed_sales#74, thu_sales#75, fri_sales#76, sat_sales#77, s_store_id#79, d_week_seq#81] +Output [8]: [d_week_seq#56 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#56, 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] (51) BroadcastExchange -Input [8]: [d_week_seq2#82, s_store_id2#83, sun_sales2#84, mon_sales2#85, wed_sales2#86, thu_sales2#87, fri_sales2#88, sat_sales2#89] +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=9] (52) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#44, d_week_seq1#43] -Right keys [2]: [s_store_id2#83, (d_week_seq2#82 - 52)] +Right keys [2]: [s_store_id2#81, (d_week_seq2#80 - 52)] Join type: Inner Join condition: None (53) Project [codegen id : 10] -Output [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (sun_sales1#45 / sun_sales2#84) AS (sun_sales1 / sun_sales2)#90, (mon_sales1#46 / mon_sales2#85) AS (mon_sales1 / mon_sales2)#91, (tue_sales1#47 / tue_sales1#47) AS (tue_sales1 / tue_sales1)#92, (wed_sales1#48 / wed_sales2#86) AS (wed_sales1 / wed_sales2)#93, (thu_sales1#49 / thu_sales2#87) AS (thu_sales1 / thu_sales2)#94, (fri_sales1#50 / fri_sales2#88) AS (fri_sales1 / fri_sales2)#95, (sat_sales1#51 / sat_sales2#89) AS (sat_sales1 / sat_sales2)#96] -Input [18]: [s_store_name1#42, d_week_seq1#43, s_store_id1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, d_week_seq2#82, s_store_id2#83, sun_sales2#84, mon_sales2#85, wed_sales2#86, thu_sales2#87, fri_sales2#88, sat_sales2#89] +Output [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (sun_sales1#45 / sun_sales2#82) AS (sun_sales1 / sun_sales2)#88, (mon_sales1#46 / mon_sales2#83) AS (mon_sales1 / mon_sales2)#89, (tue_sales1#47 / tue_sales1#47) AS (tue_sales1 / tue_sales1)#90, (wed_sales1#48 / wed_sales2#84) AS (wed_sales1 / wed_sales2)#91, (thu_sales1#49 / thu_sales2#85) AS (thu_sales1 / thu_sales2)#92, (fri_sales1#50 / fri_sales2#86) AS (fri_sales1 / fri_sales2)#93, (sat_sales1#51 / sat_sales2#87) AS (sat_sales1 / sat_sales2)#94] +Input [18]: [s_store_name1#42, d_week_seq1#43, s_store_id1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, 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] (54) TakeOrderedAndProject -Input [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (sun_sales1 / sun_sales2)#90, (mon_sales1 / mon_sales2)#91, (tue_sales1 / tue_sales1)#92, (wed_sales1 / wed_sales2)#93, (thu_sales1 / thu_sales2)#94, (fri_sales1 / fri_sales2)#95, (sat_sales1 / sat_sales2)#96] -Arguments: 100, [s_store_name1#42 ASC NULLS FIRST, s_store_id1#44 ASC NULLS FIRST, d_week_seq1#43 ASC NULLS FIRST], [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (sun_sales1 / sun_sales2)#90, (mon_sales1 / mon_sales2)#91, (tue_sales1 / tue_sales1)#92, (wed_sales1 / wed_sales2)#93, (thu_sales1 / thu_sales2)#94, (fri_sales1 / fri_sales2)#95, (sat_sales1 / sat_sales2)#96] +Input [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (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#42 ASC NULLS FIRST, s_store_id1#44 ASC NULLS FIRST, d_week_seq1#43 ASC NULLS FIRST], [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (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] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery#7, [id=#8] -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#40, d_week_seq#41] 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,1185),LessThanOrEqual(d_month_seq,1196)),And(GreaterThanOrEqual(d_month_seq,1197),LessThanOrEqual(d_month_seq,1208)))] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] @@ -339,74 +340,34 @@ Input [2]: [d_month_seq#40, d_week_seq#41] (57) Filter [codegen id : 1] Input [2]: [d_month_seq#40, d_week_seq#41] -Condition : (((isnotnull(d_month_seq#40) AND (d_month_seq#40 >= 1185)) AND (d_month_seq#40 <= 1196)) AND isnotnull(d_week_seq#41)) +Condition : ((isnotnull(d_month_seq#40) AND isnotnull(d_week_seq#41)) AND (((d_month_seq#40 >= 1185) AND (d_month_seq#40 <= 1196)) OR ((d_month_seq#40 >= 1197) AND (d_month_seq#40 <= 1208)))) (58) Project [codegen id : 1] -Output [1]: [d_week_seq#41] +Output [3]: [d_week_seq#41, ((isnotnull(d_month_seq#40) AND ((d_month_seq#40 >= 1185) AND (d_month_seq#40 <= 1196))) AND isnotnull(d_week_seq#41)) AS propagatedFilter#95, ((isnotnull(d_month_seq#40) AND ((d_month_seq#40 >= 1197) AND (d_month_seq#40 <= 1208))) AND isnotnull(d_week_seq#41)) AS propagatedFilter#96] Input [2]: [d_month_seq#40, d_week_seq#41] (59) ObjectHashAggregate -Input [1]: [d_week_seq#41] +Input [3]: [d_week_seq#41, propagatedFilter#95, propagatedFilter#96] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [buf#97] -Results [1]: [buf#98] +Functions [2]: [partial_bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0) FILTER (WHERE propagatedFilter#95), partial_bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0) FILTER (WHERE propagatedFilter#96)] +Aggregate Attributes [2]: [buf#97, buf#98] +Results [2]: [buf#99, buf#100] (60) Exchange -Input [1]: [buf#98] +Input [2]: [buf#99, buf#100] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] (61) ObjectHashAggregate -Input [1]: [buf#98] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#99] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#99 AS bloomFilter#100] - -Subquery:2 Hosting operator id = 31 Hosting Expression = Subquery scalar-subquery#58, [id=#59] -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#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,1197), LessThanOrEqual(d_month_seq,1208), IsNotNull(d_week_seq)] -ReadSchema: struct - -(63) ColumnarToRow [codegen id : 1] -Input [2]: [d_month_seq#80, d_week_seq#81] - -(64) Filter [codegen id : 1] -Input [2]: [d_month_seq#80, d_week_seq#81] -Condition : (((isnotnull(d_month_seq#80) AND (d_month_seq#80 >= 1197)) AND (d_month_seq#80 <= 1208)) AND isnotnull(d_week_seq#81)) - -(65) Project [codegen id : 1] -Output [1]: [d_week_seq#81] -Input [2]: [d_month_seq#80, d_week_seq#81] - -(66) ObjectHashAggregate -Input [1]: [d_week_seq#81] +Input [2]: [buf#99, buf#100] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#81, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [buf#101] -Results [1]: [buf#102] +Functions [2]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0), bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)] +Aggregate Attributes [2]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#101, bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#102] +Results [2]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#101 AS bloomFilter#103, bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#102 AS bloomFilter#104] -(67) Exchange -Input [1]: [buf#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] +(62) Project [codegen id : 2] +Output [1]: [named_struct(bloomFilter, bloomFilter#103, bloomFilter, bloomFilter#104) AS mergedValue#105] +Input [2]: [bloomFilter#103, bloomFilter#104] -(68) ObjectHashAggregate -Input [1]: [buf#102] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#81, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#81, 42), 335, 8990, 0, 0)#103] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#81, 42), 335, 8990, 0, 0)#103 AS bloomFilter#104] +Subquery:2 Hosting operator id = 31 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] 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..db9f90cc372f2 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,propagatedFilter] [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 91dbbbc0c8eef..1429fb77db2f5 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 @@ -94,7 +94,7 @@ Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] (10) Filter [codegen id : 3] Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : ((isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) AND might_contain(Subquery scalar-subquery#12, [id=#13], xxhash64(d_week_seq#10, 42))) +Condition : ((isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) AND might_contain(Subquery scalar-subquery#12, [id=#13].bloomFilter, xxhash64(d_week_seq#10, 42))) (11) BroadcastExchange Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] @@ -202,7 +202,7 @@ Input [3]: [d_date_sk#60, d_week_seq#61, d_day_name#62] (33) Filter [codegen id : 8] Input [3]: [d_date_sk#60, d_week_seq#61, d_day_name#62] -Condition : ((isnotnull(d_date_sk#60) AND isnotnull(d_week_seq#61)) AND might_contain(Subquery scalar-subquery#63, [id=#64], xxhash64(d_week_seq#61, 42))) +Condition : ((isnotnull(d_date_sk#60) AND isnotnull(d_week_seq#61)) AND might_contain(ReusedSubquery Subquery scalar-subquery#12, [id=#13].bloomFilter, xxhash64(d_week_seq#61, 42))) (34) BroadcastExchange Input [3]: [d_date_sk#60, d_week_seq#61, d_day_name#62] @@ -222,91 +222,92 @@ Input [5]: [sold_date_sk#54, sales_price#55, d_date_sk#60, d_week_seq#61, d_day_ Input [3]: [sales_price#55, d_week_seq#61, d_day_name#62] Keys [1]: [d_week_seq#61] Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Sunday ) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Monday ) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Tuesday ) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Wednesday) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Thursday ) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Friday ) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Saturday ) THEN sales_price#55 END))] -Aggregate Attributes [7]: [sum#65, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] -Results [8]: [d_week_seq#61, sum#72, sum#73, sum#74, sum#75, sum#76, sum#77, sum#78] +Aggregate Attributes [7]: [sum#63, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69] +Results [8]: [d_week_seq#61, sum#70, sum#71, sum#72, sum#73, sum#74, sum#75, sum#76] (38) Exchange -Input [8]: [d_week_seq#61, sum#72, sum#73, sum#74, sum#75, sum#76, sum#77, sum#78] +Input [8]: [d_week_seq#61, sum#70, sum#71, sum#72, sum#73, sum#74, sum#75, sum#76] Arguments: hashpartitioning(d_week_seq#61, 5), ENSURE_REQUIREMENTS, [plan_id=5] (39) HashAggregate [codegen id : 11] -Input [8]: [d_week_seq#61, sum#72, sum#73, sum#74, sum#75, sum#76, sum#77, sum#78] +Input [8]: [d_week_seq#61, sum#70, sum#71, sum#72, sum#73, sum#74, sum#75, sum#76] Keys [1]: [d_week_seq#61] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#62 = Sunday ) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Monday ) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Tuesday ) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Wednesday) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Thursday ) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Friday ) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Saturday ) THEN sales_price#55 END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#62 = Sunday ) THEN sales_price#55 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Monday ) THEN sales_price#55 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Tuesday ) THEN sales_price#55 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Wednesday) THEN sales_price#55 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Thursday ) THEN sales_price#55 END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Friday ) THEN sales_price#55 END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Saturday ) THEN sales_price#55 END))#34] -Results [8]: [d_week_seq#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Sunday ) THEN sales_price#55 END))#28,17,2) AS sun_sales#79, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Monday ) THEN sales_price#55 END))#29,17,2) AS mon_sales#80, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Tuesday ) THEN sales_price#55 END))#30,17,2) AS tue_sales#81, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Wednesday) THEN sales_price#55 END))#31,17,2) AS wed_sales#82, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Thursday ) THEN sales_price#55 END))#32,17,2) AS thu_sales#83, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Friday ) THEN sales_price#55 END))#33,17,2) AS fri_sales#84, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Saturday ) THEN sales_price#55 END))#34,17,2) AS sat_sales#85] +Results [8]: [d_week_seq#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Sunday ) THEN sales_price#55 END))#28,17,2) AS sun_sales#77, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Monday ) THEN sales_price#55 END))#29,17,2) AS mon_sales#78, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Tuesday ) THEN sales_price#55 END))#30,17,2) AS tue_sales#79, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Wednesday) THEN sales_price#55 END))#31,17,2) AS wed_sales#80, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Thursday ) THEN sales_price#55 END))#32,17,2) AS thu_sales#81, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Friday ) THEN sales_price#55 END))#33,17,2) AS fri_sales#82, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Saturday ) THEN sales_price#55 END))#34,17,2) AS sat_sales#83] (40) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#86, d_year#87] +Output [2]: [d_week_seq#84, d_year#85] 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 (41) ColumnarToRow [codegen id : 10] -Input [2]: [d_week_seq#86, d_year#87] +Input [2]: [d_week_seq#84, d_year#85] (42) Filter [codegen id : 10] -Input [2]: [d_week_seq#86, d_year#87] -Condition : ((isnotnull(d_year#87) AND (d_year#87 = 2002)) AND isnotnull(d_week_seq#86)) +Input [2]: [d_week_seq#84, d_year#85] +Condition : ((isnotnull(d_year#85) AND (d_year#85 = 2002)) AND isnotnull(d_week_seq#84)) (43) Project [codegen id : 10] -Output [1]: [d_week_seq#86] -Input [2]: [d_week_seq#86, d_year#87] +Output [1]: [d_week_seq#84] +Input [2]: [d_week_seq#84, d_year#85] (44) BroadcastExchange -Input [1]: [d_week_seq#86] +Input [1]: [d_week_seq#84] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] (45) BroadcastHashJoin [codegen id : 11] Left keys [1]: [d_week_seq#61] -Right keys [1]: [d_week_seq#86] +Right keys [1]: [d_week_seq#84] Join type: Inner Join condition: None (46) Project [codegen id : 11] -Output [8]: [d_week_seq#61 AS d_week_seq2#88, sun_sales#79 AS sun_sales2#89, mon_sales#80 AS mon_sales2#90, tue_sales#81 AS tue_sales2#91, wed_sales#82 AS wed_sales2#92, thu_sales#83 AS thu_sales2#93, fri_sales#84 AS fri_sales2#94, sat_sales#85 AS sat_sales2#95] -Input [9]: [d_week_seq#61, sun_sales#79, mon_sales#80, tue_sales#81, wed_sales#82, thu_sales#83, fri_sales#84, sat_sales#85, d_week_seq#86] +Output [8]: [d_week_seq#61 AS d_week_seq2#86, sun_sales#77 AS sun_sales2#87, mon_sales#78 AS mon_sales2#88, tue_sales#79 AS tue_sales2#89, wed_sales#80 AS wed_sales2#90, thu_sales#81 AS thu_sales2#91, fri_sales#82 AS fri_sales2#92, sat_sales#83 AS sat_sales2#93] +Input [9]: [d_week_seq#61, sun_sales#77, mon_sales#78, tue_sales#79, wed_sales#80, thu_sales#81, fri_sales#82, sat_sales#83, d_week_seq#84] (47) BroadcastExchange -Input [8]: [d_week_seq2#88, sun_sales2#89, mon_sales2#90, tue_sales2#91, wed_sales2#92, thu_sales2#93, fri_sales2#94, sat_sales2#95] +Input [8]: [d_week_seq2#86, sun_sales2#87, mon_sales2#88, tue_sales2#89, wed_sales2#90, thu_sales2#91, fri_sales2#92, sat_sales2#93] Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=7] (48) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#44] -Right keys [1]: [(d_week_seq2#88 - 53)] +Right keys [1]: [(d_week_seq2#86 - 53)] Join type: Inner Join condition: None (49) Project [codegen id : 12] -Output [8]: [d_week_seq1#44, round((sun_sales1#45 / sun_sales2#89), 2) AS round((sun_sales1 / sun_sales2), 2)#96, round((mon_sales1#46 / mon_sales2#90), 2) AS round((mon_sales1 / mon_sales2), 2)#97, round((tue_sales1#47 / tue_sales2#91), 2) AS round((tue_sales1 / tue_sales2), 2)#98, round((wed_sales1#48 / wed_sales2#92), 2) AS round((wed_sales1 / wed_sales2), 2)#99, round((thu_sales1#49 / thu_sales2#93), 2) AS round((thu_sales1 / thu_sales2), 2)#100, round((fri_sales1#50 / fri_sales2#94), 2) AS round((fri_sales1 / fri_sales2), 2)#101, round((sat_sales1#51 / sat_sales2#95), 2) AS round((sat_sales1 / sat_sales2), 2)#102] -Input [16]: [d_week_seq1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, d_week_seq2#88, sun_sales2#89, mon_sales2#90, tue_sales2#91, wed_sales2#92, thu_sales2#93, fri_sales2#94, sat_sales2#95] +Output [8]: [d_week_seq1#44, round((sun_sales1#45 / sun_sales2#87), 2) AS round((sun_sales1 / sun_sales2), 2)#94, round((mon_sales1#46 / mon_sales2#88), 2) AS round((mon_sales1 / mon_sales2), 2)#95, round((tue_sales1#47 / tue_sales2#89), 2) AS round((tue_sales1 / tue_sales2), 2)#96, round((wed_sales1#48 / wed_sales2#90), 2) AS round((wed_sales1 / wed_sales2), 2)#97, round((thu_sales1#49 / thu_sales2#91), 2) AS round((thu_sales1 / thu_sales2), 2)#98, round((fri_sales1#50 / fri_sales2#92), 2) AS round((fri_sales1 / fri_sales2), 2)#99, round((sat_sales1#51 / sat_sales2#93), 2) AS round((sat_sales1 / sat_sales2), 2)#100] +Input [16]: [d_week_seq1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, d_week_seq2#86, sun_sales2#87, mon_sales2#88, tue_sales2#89, wed_sales2#90, thu_sales2#91, fri_sales2#92, sat_sales2#93] (50) Exchange -Input [8]: [d_week_seq1#44, round((sun_sales1 / sun_sales2), 2)#96, round((mon_sales1 / mon_sales2), 2)#97, round((tue_sales1 / tue_sales2), 2)#98, round((wed_sales1 / wed_sales2), 2)#99, round((thu_sales1 / thu_sales2), 2)#100, round((fri_sales1 / fri_sales2), 2)#101, round((sat_sales1 / sat_sales2), 2)#102] +Input [8]: [d_week_seq1#44, round((sun_sales1 / sun_sales2), 2)#94, round((mon_sales1 / mon_sales2), 2)#95, round((tue_sales1 / tue_sales2), 2)#96, round((wed_sales1 / wed_sales2), 2)#97, round((thu_sales1 / thu_sales2), 2)#98, round((fri_sales1 / fri_sales2), 2)#99, round((sat_sales1 / sat_sales2), 2)#100] Arguments: rangepartitioning(d_week_seq1#44 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=8] (51) Sort [codegen id : 13] -Input [8]: [d_week_seq1#44, round((sun_sales1 / sun_sales2), 2)#96, round((mon_sales1 / mon_sales2), 2)#97, round((tue_sales1 / tue_sales2), 2)#98, round((wed_sales1 / wed_sales2), 2)#99, round((thu_sales1 / thu_sales2), 2)#100, round((fri_sales1 / fri_sales2), 2)#101, round((sat_sales1 / sat_sales2), 2)#102] +Input [8]: [d_week_seq1#44, round((sun_sales1 / sun_sales2), 2)#94, round((mon_sales1 / mon_sales2), 2)#95, round((tue_sales1 / tue_sales2), 2)#96, round((wed_sales1 / wed_sales2), 2)#97, round((thu_sales1 / thu_sales2), 2)#98, round((fri_sales1 / fri_sales2), 2)#99, round((sat_sales1 / sat_sales2), 2)#100] Arguments: [d_week_seq1#44 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 10 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -ObjectHashAggregate (58) -+- Exchange (57) - +- ObjectHashAggregate (56) - +- * Project (55) - +- * Filter (54) - +- * ColumnarToRow (53) - +- Scan parquet spark_catalog.default.date_dim (52) +* Project (59) ++- ObjectHashAggregate (58) + +- Exchange (57) + +- ObjectHashAggregate (56) + +- * Project (55) + +- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet spark_catalog.default.date_dim (52) (52) Scan parquet spark_catalog.default.date_dim Output [2]: [d_week_seq#42, d_year#43] 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,2001),EqualTo(d_year,2002))] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] @@ -314,74 +315,34 @@ Input [2]: [d_week_seq#42, d_year#43] (54) Filter [codegen id : 1] Input [2]: [d_week_seq#42, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2001)) AND isnotnull(d_week_seq#42)) +Condition : ((isnotnull(d_year#43) AND isnotnull(d_week_seq#42)) AND ((d_year#43 = 2001) OR (d_year#43 = 2002))) (55) Project [codegen id : 1] -Output [1]: [d_week_seq#42] +Output [3]: [d_week_seq#42, ((isnotnull(d_year#43) AND (d_year#43 = 2001)) AND isnotnull(d_week_seq#42)) AS propagatedFilter#101, ((isnotnull(d_year#43) AND (d_year#43 = 2002)) AND isnotnull(d_week_seq#42)) AS propagatedFilter#102] Input [2]: [d_week_seq#42, d_year#43] (56) ObjectHashAggregate -Input [1]: [d_week_seq#42] +Input [3]: [d_week_seq#42, propagatedFilter#101, propagatedFilter#102] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [buf#103] -Results [1]: [buf#104] +Functions [2]: [partial_bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0) FILTER (WHERE propagatedFilter#101), partial_bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0) FILTER (WHERE propagatedFilter#102)] +Aggregate Attributes [2]: [buf#103, buf#104] +Results [2]: [buf#105, buf#106] (57) Exchange -Input [1]: [buf#104] +Input [2]: [buf#105, buf#106] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] (58) ObjectHashAggregate -Input [1]: [buf#104] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)#105] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)#105 AS bloomFilter#106] - -Subquery:2 Hosting operator id = 33 Hosting Expression = Subquery scalar-subquery#63, [id=#64] -ObjectHashAggregate (65) -+- Exchange (64) - +- ObjectHashAggregate (63) - +- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet spark_catalog.default.date_dim (59) - - -(59) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#86, d_year#87] -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 - -(60) ColumnarToRow [codegen id : 1] -Input [2]: [d_week_seq#86, d_year#87] - -(61) Filter [codegen id : 1] -Input [2]: [d_week_seq#86, d_year#87] -Condition : ((isnotnull(d_year#87) AND (d_year#87 = 2002)) AND isnotnull(d_week_seq#86)) - -(62) Project [codegen id : 1] -Output [1]: [d_week_seq#86] -Input [2]: [d_week_seq#86, d_year#87] - -(63) ObjectHashAggregate -Input [1]: [d_week_seq#86] +Input [2]: [buf#105, buf#106] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#86, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [buf#107] -Results [1]: [buf#108] +Functions [2]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0), bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)] +Aggregate Attributes [2]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)#107, bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)#108] +Results [2]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)#107 AS bloomFilter#109, bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)#108 AS bloomFilter#110] -(64) Exchange -Input [1]: [buf#108] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] +(59) Project [codegen id : 2] +Output [1]: [named_struct(bloomFilter, bloomFilter#109, bloomFilter, bloomFilter#110) AS mergedValue#111] +Input [2]: [bloomFilter#109, bloomFilter#110] -(65) ObjectHashAggregate -Input [1]: [buf#108] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#86, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#86, 42), 362, 9656, 0, 0)#109] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#86, 42), 362, 9656, 0, 0)#109 AS bloomFilter#110] +Subquery:2 Hosting operator id = 33 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] 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 4fb858b42521a..48ec512c4c542 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 @@ -31,15 +31,18 @@ WholeStageCodegen (13) WholeStageCodegen (3) 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,propagatedFilter] [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] @@ -79,21 +82,12 @@ WholeStageCodegen (13) BroadcastExchange #8 WholeStageCodegen (8) 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] InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (10) 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/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt index 2f020e12b3fc5..a5a408b4d1922 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=#8], 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=#8].bloomFilter, xxhash64(d_week_seq#5, 42))) (7) BroadcastExchange Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] @@ -203,7 +203,7 @@ Input [3]: [d_date_sk#55, d_week_seq#56, d_day_name#57] (31) Filter [codegen id : 5] Input [3]: [d_date_sk#55, d_week_seq#56, d_day_name#57] -Condition : ((isnotnull(d_date_sk#55) AND isnotnull(d_week_seq#56)) AND might_contain(Subquery scalar-subquery#58, [id=#59], xxhash64(d_week_seq#56, 42))) +Condition : ((isnotnull(d_date_sk#55) AND isnotnull(d_week_seq#56)) AND might_contain(ReusedSubquery Subquery scalar-subquery#7, [id=#8].bloomFilter, xxhash64(d_week_seq#56, 42))) (32) BroadcastExchange Input [3]: [d_date_sk#55, d_week_seq#56, d_day_name#57] @@ -223,115 +223,116 @@ Input [6]: [ss_store_sk#52, ss_sales_price#53, ss_sold_date_sk#54, d_date_sk#55, Input [4]: [ss_store_sk#52, ss_sales_price#53, d_week_seq#56, d_day_name#57] Keys [2]: [d_week_seq#56, ss_store_sk#52] Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Sunday ) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Monday ) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Tuesday ) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Wednesday) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Thursday ) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Friday ) THEN ss_sales_price#53 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#57 = Saturday ) THEN ss_sales_price#53 END))] -Aggregate Attributes [7]: [sum#60, sum#61, sum#62, sum#63, sum#64, sum#65, sum#66] -Results [9]: [d_week_seq#56, ss_store_sk#52, sum#67, sum#68, sum#69, sum#70, sum#71, sum#72, sum#73] +Aggregate Attributes [7]: [sum#58, sum#59, sum#60, sum#61, sum#62, sum#63, sum#64] +Results [9]: [d_week_seq#56, ss_store_sk#52, sum#65, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] (36) Exchange -Input [9]: [d_week_seq#56, ss_store_sk#52, sum#67, sum#68, sum#69, sum#70, sum#71, sum#72, sum#73] +Input [9]: [d_week_seq#56, ss_store_sk#52, sum#65, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] Arguments: hashpartitioning(d_week_seq#56, ss_store_sk#52, 5), ENSURE_REQUIREMENTS, [plan_id=6] (37) HashAggregate [codegen id : 9] -Input [9]: [d_week_seq#56, ss_store_sk#52, sum#67, sum#68, sum#69, sum#70, sum#71, sum#72, sum#73] +Input [9]: [d_week_seq#56, ss_store_sk#52, sum#65, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] Keys [2]: [d_week_seq#56, ss_store_sk#52] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#57 = Sunday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Monday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Tuesday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Wednesday) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Thursday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Friday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#57 = Saturday ) THEN ss_sales_price#53 END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#57 = Sunday ) THEN ss_sales_price#53 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Monday ) THEN ss_sales_price#53 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Tuesday ) THEN ss_sales_price#53 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Wednesday) THEN ss_sales_price#53 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Thursday ) THEN ss_sales_price#53 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Friday ) THEN ss_sales_price#53 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#57 = Saturday ) THEN ss_sales_price#53 END))#29] -Results [9]: [d_week_seq#56, ss_store_sk#52, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Sunday ) THEN ss_sales_price#53 END))#23,17,2) AS sun_sales#74, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Monday ) THEN ss_sales_price#53 END))#24,17,2) AS mon_sales#75, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Tuesday ) THEN ss_sales_price#53 END))#25,17,2) AS tue_sales#76, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Wednesday) THEN ss_sales_price#53 END))#26,17,2) AS wed_sales#77, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Thursday ) THEN ss_sales_price#53 END))#27,17,2) AS thu_sales#78, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Friday ) THEN ss_sales_price#53 END))#28,17,2) AS fri_sales#79, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Saturday ) THEN ss_sales_price#53 END))#29,17,2) AS sat_sales#80] +Results [9]: [d_week_seq#56, ss_store_sk#52, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Sunday ) THEN ss_sales_price#53 END))#23,17,2) AS sun_sales#72, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Monday ) THEN ss_sales_price#53 END))#24,17,2) AS mon_sales#73, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Tuesday ) THEN ss_sales_price#53 END))#25,17,2) AS tue_sales#74, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Wednesday) THEN ss_sales_price#53 END))#26,17,2) AS wed_sales#75, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Thursday ) THEN ss_sales_price#53 END))#27,17,2) AS thu_sales#76, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Friday ) THEN ss_sales_price#53 END))#28,17,2) AS fri_sales#77, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#57 = Saturday ) THEN ss_sales_price#53 END))#29,17,2) AS sat_sales#78] (38) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#81, s_store_id#82] +Output [2]: [s_store_sk#79, s_store_id#80] 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#81, s_store_id#82] +Input [2]: [s_store_sk#79, s_store_id#80] (40) Filter [codegen id : 7] -Input [2]: [s_store_sk#81, s_store_id#82] -Condition : (isnotnull(s_store_sk#81) AND isnotnull(s_store_id#82)) +Input [2]: [s_store_sk#79, s_store_id#80] +Condition : (isnotnull(s_store_sk#79) AND isnotnull(s_store_id#80)) (41) BroadcastExchange -Input [2]: [s_store_sk#81, s_store_id#82] +Input [2]: [s_store_sk#79, s_store_id#80] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#52] -Right keys [1]: [s_store_sk#81] +Right keys [1]: [s_store_sk#79] Join type: Inner Join condition: None (43) Project [codegen id : 9] -Output [9]: [d_week_seq#56, sun_sales#74, mon_sales#75, tue_sales#76, wed_sales#77, thu_sales#78, fri_sales#79, sat_sales#80, s_store_id#82] -Input [11]: [d_week_seq#56, ss_store_sk#52, sun_sales#74, mon_sales#75, tue_sales#76, wed_sales#77, thu_sales#78, fri_sales#79, sat_sales#80, s_store_sk#81, s_store_id#82] +Output [9]: [d_week_seq#56, 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#56, ss_store_sk#52, 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] (44) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#83, d_week_seq#84] +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 (45) ColumnarToRow [codegen id : 8] -Input [2]: [d_month_seq#83, d_week_seq#84] +Input [2]: [d_month_seq#81, d_week_seq#82] (46) Filter [codegen id : 8] -Input [2]: [d_month_seq#83, d_week_seq#84] -Condition : (((isnotnull(d_month_seq#83) AND (d_month_seq#83 >= 1224)) AND (d_month_seq#83 <= 1235)) AND isnotnull(d_week_seq#84)) +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)) (47) Project [codegen id : 8] -Output [1]: [d_week_seq#84] -Input [2]: [d_month_seq#83, d_week_seq#84] +Output [1]: [d_week_seq#82] +Input [2]: [d_month_seq#81, d_week_seq#82] (48) BroadcastExchange -Input [1]: [d_week_seq#84] +Input [1]: [d_week_seq#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] (49) BroadcastHashJoin [codegen id : 9] Left keys [1]: [d_week_seq#56] -Right keys [1]: [d_week_seq#84] +Right keys [1]: [d_week_seq#82] Join type: Inner Join condition: None (50) Project [codegen id : 9] -Output [9]: [d_week_seq#56 AS d_week_seq2#85, s_store_id#82 AS s_store_id2#86, sun_sales#74 AS sun_sales2#87, mon_sales#75 AS mon_sales2#88, tue_sales#76 AS tue_sales2#89, wed_sales#77 AS wed_sales2#90, thu_sales#78 AS thu_sales2#91, fri_sales#79 AS fri_sales2#92, sat_sales#80 AS sat_sales2#93] -Input [10]: [d_week_seq#56, sun_sales#74, mon_sales#75, tue_sales#76, wed_sales#77, thu_sales#78, fri_sales#79, sat_sales#80, s_store_id#82, d_week_seq#84] +Output [9]: [d_week_seq#56 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#56, 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] (51) BroadcastExchange -Input [9]: [d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, tue_sales2#89, wed_sales2#90, thu_sales2#91, fri_sales2#92, sat_sales2#93] +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=9] (52) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#44, d_week_seq1#43] -Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 52)] +Right keys [2]: [s_store_id2#84, (d_week_seq2#83 - 52)] Join type: Inner Join condition: None (53) Project [codegen id : 10] -Output [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (sun_sales1#45 / sun_sales2#87) AS (sun_sales1 / sun_sales2)#94, (mon_sales1#46 / mon_sales2#88) AS (mon_sales1 / mon_sales2)#95, (tue_sales1#47 / tue_sales2#89) AS (tue_sales1 / tue_sales2)#96, (wed_sales1#48 / wed_sales2#90) AS (wed_sales1 / wed_sales2)#97, (thu_sales1#49 / thu_sales2#91) AS (thu_sales1 / thu_sales2)#98, (fri_sales1#50 / fri_sales2#92) AS (fri_sales1 / fri_sales2)#99, (sat_sales1#51 / sat_sales2#93) AS (sat_sales1 / sat_sales2)#100] -Input [19]: [s_store_name1#42, d_week_seq1#43, s_store_id1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, tue_sales2#89, wed_sales2#90, thu_sales2#91, fri_sales2#92, sat_sales2#93] +Output [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (sun_sales1#45 / sun_sales2#85) AS (sun_sales1 / sun_sales2)#92, (mon_sales1#46 / mon_sales2#86) AS (mon_sales1 / mon_sales2)#93, (tue_sales1#47 / tue_sales2#87) AS (tue_sales1 / tue_sales2)#94, (wed_sales1#48 / wed_sales2#88) AS (wed_sales1 / wed_sales2)#95, (thu_sales1#49 / thu_sales2#89) AS (thu_sales1 / thu_sales2)#96, (fri_sales1#50 / fri_sales2#90) AS (fri_sales1 / fri_sales2)#97, (sat_sales1#51 / sat_sales2#91) AS (sat_sales1 / sat_sales2)#98] +Input [19]: [s_store_name1#42, d_week_seq1#43, s_store_id1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, 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] (54) TakeOrderedAndProject -Input [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (sun_sales1 / sun_sales2)#94, (mon_sales1 / mon_sales2)#95, (tue_sales1 / tue_sales2)#96, (wed_sales1 / wed_sales2)#97, (thu_sales1 / thu_sales2)#98, (fri_sales1 / fri_sales2)#99, (sat_sales1 / sat_sales2)#100] -Arguments: 100, [s_store_name1#42 ASC NULLS FIRST, s_store_id1#44 ASC NULLS FIRST, d_week_seq1#43 ASC NULLS FIRST], [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (sun_sales1 / sun_sales2)#94, (mon_sales1 / mon_sales2)#95, (tue_sales1 / tue_sales2)#96, (wed_sales1 / wed_sales2)#97, (thu_sales1 / thu_sales2)#98, (fri_sales1 / fri_sales2)#99, (sat_sales1 / sat_sales2)#100] +Input [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (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#42 ASC NULLS FIRST, s_store_id1#44 ASC NULLS FIRST, d_week_seq1#43 ASC NULLS FIRST], [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (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] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery#7, [id=#8] -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#40, d_week_seq#41] 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,1212),LessThanOrEqual(d_month_seq,1223)),And(GreaterThanOrEqual(d_month_seq,1224),LessThanOrEqual(d_month_seq,1235)))] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] @@ -339,74 +340,34 @@ Input [2]: [d_month_seq#40, d_week_seq#41] (57) Filter [codegen id : 1] Input [2]: [d_month_seq#40, d_week_seq#41] -Condition : (((isnotnull(d_month_seq#40) AND (d_month_seq#40 >= 1212)) AND (d_month_seq#40 <= 1223)) AND isnotnull(d_week_seq#41)) +Condition : ((isnotnull(d_month_seq#40) AND isnotnull(d_week_seq#41)) AND (((d_month_seq#40 >= 1212) AND (d_month_seq#40 <= 1223)) OR ((d_month_seq#40 >= 1224) AND (d_month_seq#40 <= 1235)))) (58) Project [codegen id : 1] -Output [1]: [d_week_seq#41] +Output [3]: [d_week_seq#41, ((isnotnull(d_month_seq#40) AND ((d_month_seq#40 >= 1212) AND (d_month_seq#40 <= 1223))) AND isnotnull(d_week_seq#41)) AS propagatedFilter#99, ((isnotnull(d_month_seq#40) AND ((d_month_seq#40 >= 1224) AND (d_month_seq#40 <= 1235))) AND isnotnull(d_week_seq#41)) AS propagatedFilter#100] Input [2]: [d_month_seq#40, d_week_seq#41] (59) ObjectHashAggregate -Input [1]: [d_week_seq#41] +Input [3]: [d_week_seq#41, propagatedFilter#99, propagatedFilter#100] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [buf#101] -Results [1]: [buf#102] +Functions [2]: [partial_bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0) FILTER (WHERE propagatedFilter#99), partial_bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0) FILTER (WHERE propagatedFilter#100)] +Aggregate Attributes [2]: [buf#101, buf#102] +Results [2]: [buf#103, buf#104] (60) Exchange -Input [1]: [buf#102] +Input [2]: [buf#103, buf#104] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] (61) ObjectHashAggregate -Input [1]: [buf#102] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#103] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#103 AS bloomFilter#104] - -Subquery:2 Hosting operator id = 31 Hosting Expression = Subquery scalar-subquery#58, [id=#59] -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#83, d_week_seq#84] -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#83, d_week_seq#84] - -(64) Filter [codegen id : 1] -Input [2]: [d_month_seq#83, d_week_seq#84] -Condition : (((isnotnull(d_month_seq#83) AND (d_month_seq#83 >= 1224)) AND (d_month_seq#83 <= 1235)) AND isnotnull(d_week_seq#84)) - -(65) Project [codegen id : 1] -Output [1]: [d_week_seq#84] -Input [2]: [d_month_seq#83, d_week_seq#84] - -(66) ObjectHashAggregate -Input [1]: [d_week_seq#84] +Input [2]: [buf#103, buf#104] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#84, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [buf#105] -Results [1]: [buf#106] +Functions [2]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0), bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)] +Aggregate Attributes [2]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#105, bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#106] +Results [2]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#105 AS bloomFilter#107, bloom_filter_agg(xxhash64(d_week_seq#41, 42), 335, 8990, 0, 0)#106 AS bloomFilter#108] -(67) Exchange -Input [1]: [buf#106] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] +(62) Project [codegen id : 2] +Output [1]: [named_struct(bloomFilter, bloomFilter#107, bloomFilter, bloomFilter#108) AS mergedValue#109] +Input [2]: [bloomFilter#107, bloomFilter#108] -(68) ObjectHashAggregate -Input [1]: [buf#106] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#84, 42), 335, 8990, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#84, 42), 335, 8990, 0, 0)#107] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#84, 42), 335, 8990, 0, 0)#107 AS bloomFilter#108] +Subquery:2 Hosting operator id = 31 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] 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..d5d9776981387 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,propagatedFilter] [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 e32db674082ee..15d841569e824 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=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] +Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#3].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket2#5, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#3].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket3#6, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#3].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket4#7, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#3].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket5#8] 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#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Output [4]: [ss_quantity#9, ss_ext_discount_amt#10, ss_net_paid#11, ss_sold_date_sk#12] 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(Or(Or(Or(Or(Or(Or(Or(Or(And(GreaterThanOrEqual(ss_quantity,1),LessThanOrEqual(ss_quantity,20)),And(GreaterThanOrEqual(ss_quantity,21),LessThanOrEqual(ss_quantity,40))),And(GreaterThanOrEqual(ss_quantity,21),LessThanOrEqual(ss_quantity,40))),And(GreaterThanOrEqual(ss_quantity,21),LessThanOrEqual(ss_quantity,40))),And(GreaterThanOrEqual(ss_quantity,41),LessThanOrEqual(ss_quantity,60))),And(GreaterThanOrEqual(ss_quantity,41),LessThanOrEqual(ss_quantity,60))),And(GreaterThanOrEqual(ss_quantity,41),LessThanOrEqual(ss_quantity,60))),And(GreaterThanOrEqual(ss_quantity,61),LessThanOrEqual(ss_quantity,80))),And(GreaterThanOrEqual(ss_quantity,61),LessThanOrEqual(ss_quantity,80))),And(GreaterThanOrEqual(ss_quantity,61),LessThanOrEqual(ss_quantity,80))),And(GreaterThanOrEqual(ss_quantity,81),LessThanOrEqual(ss_quantity,100))),And(GreaterThanOrEqual(ss_quantity,81),LessThanOrEqual(ss_quantity,100))),And(GreaterThanOrEqual(ss_quantity,81),LessThanOrEqual(ss_quantity,100)))] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Input [4]: [ss_quantity#9, ss_ext_discount_amt#10, ss_net_paid#11, ss_sold_date_sk#12] (7) Filter [codegen id : 1] -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) +Input [4]: [ss_quantity#9, ss_ext_discount_amt#10, ss_net_paid#11, ss_sold_date_sk#12] +Condition : (isnotnull(ss_quantity#9) AND ((((((((((((((ss_quantity#9 >= 1) AND (ss_quantity#9 <= 20)) OR ((ss_quantity#9 >= 21) AND (ss_quantity#9 <= 40))) OR ((ss_quantity#9 >= 21) AND (ss_quantity#9 <= 40))) OR ((ss_quantity#9 >= 21) AND (ss_quantity#9 <= 40))) OR ((ss_quantity#9 >= 41) AND (ss_quantity#9 <= 60))) OR ((ss_quantity#9 >= 41) AND (ss_quantity#9 <= 60))) OR ((ss_quantity#9 >= 41) AND (ss_quantity#9 <= 60))) OR ((ss_quantity#9 >= 61) AND (ss_quantity#9 <= 80))) OR ((ss_quantity#9 >= 61) AND (ss_quantity#9 <= 80))) OR ((ss_quantity#9 >= 61) AND (ss_quantity#9 <= 80))) OR ((ss_quantity#9 >= 81) AND (ss_quantity#9 <= 100))) OR ((ss_quantity#9 >= 81) AND (ss_quantity#9 <= 100))) OR ((ss_quantity#9 >= 81) AND (ss_quantity#9 <= 100)))) (8) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Output [7]: [ss_ext_discount_amt#10, ss_net_paid#11, (isnotnull(ss_quantity#9) AND ((ss_quantity#9 >= 1) AND (ss_quantity#9 <= 20))) AS propagatedFilter#13, (isnotnull(ss_quantity#9) AND ((ss_quantity#9 >= 21) AND (ss_quantity#9 <= 40))) AS propagatedFilter#14, (isnotnull(ss_quantity#9) AND ((ss_quantity#9 >= 41) AND (ss_quantity#9 <= 60))) AS propagatedFilter#15, (isnotnull(ss_quantity#9) AND ((ss_quantity#9 >= 61) AND (ss_quantity#9 <= 80))) AS propagatedFilter#16, (isnotnull(ss_quantity#9) AND ((ss_quantity#9 >= 81) AND (ss_quantity#9 <= 100))) AS propagatedFilter#17] +Input [4]: [ss_quantity#9, ss_ext_discount_amt#10, ss_net_paid#11, ss_sold_date_sk#12] (9) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] +Input [7]: [ss_ext_discount_amt#10, ss_net_paid#11, propagatedFilter#13, propagatedFilter#14, propagatedFilter#15, propagatedFilter#16, propagatedFilter#17] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] -Aggregate Attributes [5]: [count#21, sum#22, count#23, sum#24, count#25] -Results [5]: [count#26, sum#27, count#28, sum#29, count#30] +Functions [15]: [partial_count(1) FILTER (WHERE propagatedFilter#13), partial_avg(UnscaledValue(ss_ext_discount_amt#10)) FILTER (WHERE propagatedFilter#13), partial_avg(UnscaledValue(ss_net_paid#11)) FILTER (WHERE propagatedFilter#13), partial_count(1) FILTER (WHERE propagatedFilter#14), partial_avg(UnscaledValue(ss_ext_discount_amt#10)) FILTER (WHERE propagatedFilter#14), partial_avg(UnscaledValue(ss_net_paid#11)) FILTER (WHERE propagatedFilter#14), partial_count(1) FILTER (WHERE propagatedFilter#15), partial_avg(UnscaledValue(ss_ext_discount_amt#10)) FILTER (WHERE propagatedFilter#15), partial_avg(UnscaledValue(ss_net_paid#11)) FILTER (WHERE propagatedFilter#15), partial_count(1) FILTER (WHERE propagatedFilter#16), partial_avg(UnscaledValue(ss_ext_discount_amt#10)) FILTER (WHERE propagatedFilter#16), partial_avg(UnscaledValue(ss_net_paid#11)) FILTER (WHERE propagatedFilter#16), partial_count(1) FILTER (WHERE propagatedFilter#17), partial_avg(UnscaledValue(ss_ext_discount_amt#10)) FILTER (WHERE propagatedFilter#17), partial_avg(UnscaledValue(ss_net_paid#11)) FILTER (WHERE propagatedFilter#17)] +Aggregate Attributes [25]: [count#18, sum#19, count#20, sum#21, count#22, count#23, sum#24, count#25, sum#26, count#27, count#28, sum#29, count#30, sum#31, count#32, count#33, sum#34, count#35, sum#36, count#37, count#38, sum#39, count#40, sum#41, count#42] +Results [25]: [count#43, sum#44, count#45, sum#46, count#47, count#48, sum#49, count#50, sum#51, count#52, count#53, sum#54, count#55, sum#56, count#57, count#58, sum#59, count#60, sum#61, count#62, count#63, sum#64, count#65, sum#66, count#67] (10) Exchange -Input [5]: [count#26, sum#27, count#28, sum#29, count#30] +Input [25]: [count#43, sum#44, count#45, sum#46, count#47, count#48, sum#49, count#50, sum#51, count#52, count#53, sum#54, count#55, sum#56, count#57, count#58, sum#59, count#60, sum#61, count#62, count#63, sum#64, count#65, sum#66, count#67] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] (11) HashAggregate [codegen id : 2] -Input [5]: [count#26, sum#27, count#28, sum#29, count#30] +Input [25]: [count#43, sum#44, count#45, sum#46, count#47, count#48, sum#49, count#50, sum#51, count#52, count#53, sum#54, count#55, sum#56, count#57, count#58, sum#59, count#60, sum#61, count#62, count#63, sum#64, count#65, sum#66, count#67] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] -Aggregate Attributes [3]: [count(1)#31, avg(UnscaledValue(ss_ext_discount_amt#18))#32, avg(UnscaledValue(ss_net_paid#19))#33] -Results [3]: [count(1)#31 AS count(1)#34, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#32 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#35, cast((avg(UnscaledValue(ss_net_paid#19))#33 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#36] +Functions [15]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#10)), avg(UnscaledValue(ss_net_paid#11)), count(1), avg(UnscaledValue(ss_ext_discount_amt#10)), avg(UnscaledValue(ss_net_paid#11)), count(1), avg(UnscaledValue(ss_ext_discount_amt#10)), avg(UnscaledValue(ss_net_paid#11)), count(1), avg(UnscaledValue(ss_ext_discount_amt#10)), avg(UnscaledValue(ss_net_paid#11)), count(1), avg(UnscaledValue(ss_ext_discount_amt#10)), avg(UnscaledValue(ss_net_paid#11))] +Aggregate Attributes [15]: [count(1)#68, avg(UnscaledValue(ss_ext_discount_amt#10))#69, avg(UnscaledValue(ss_net_paid#11))#70, count(1)#71, avg(UnscaledValue(ss_ext_discount_amt#10))#72, avg(UnscaledValue(ss_net_paid#11))#73, count(1)#74, avg(UnscaledValue(ss_ext_discount_amt#10))#75, avg(UnscaledValue(ss_net_paid#11))#76, count(1)#77, avg(UnscaledValue(ss_ext_discount_amt#10))#78, avg(UnscaledValue(ss_net_paid#11))#79, count(1)#80, avg(UnscaledValue(ss_ext_discount_amt#10))#81, avg(UnscaledValue(ss_net_paid#11))#82] +Results [15]: [count(1)#68 AS count(1)#83, cast((avg(UnscaledValue(ss_ext_discount_amt#10))#69 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#84, cast((avg(UnscaledValue(ss_net_paid#11))#70 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#85, count(1)#71 AS count(1)#86, cast((avg(UnscaledValue(ss_ext_discount_amt#10))#72 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#87, cast((avg(UnscaledValue(ss_net_paid#11))#73 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#88, count(1)#74 AS count(1)#89, cast((avg(UnscaledValue(ss_ext_discount_amt#10))#75 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90, cast((avg(UnscaledValue(ss_net_paid#11))#76 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#91, count(1)#77 AS count(1)#92, cast((avg(UnscaledValue(ss_ext_discount_amt#10))#78 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#93, cast((avg(UnscaledValue(ss_net_paid#11))#79 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#94, count(1)#80 AS count(1)#95, cast((avg(UnscaledValue(ss_ext_discount_amt#10))#81 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#96, cast((avg(UnscaledValue(ss_net_paid#11))#82 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] (12) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#34, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#35, avg(ss_net_paid), avg(ss_net_paid)#36) AS mergedValue#37] -Input [3]: [count(1)#34, avg(ss_ext_discount_amt)#35, avg(ss_net_paid)#36] +Output [1]: [named_struct(count(1), count(1)#83, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#84, avg(ss_net_paid), avg(ss_net_paid)#85, count(1), count(1)#86, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#87, avg(ss_net_paid), avg(ss_net_paid)#88, count(1), count(1)#89, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#90, avg(ss_net_paid), avg(ss_net_paid)#91, 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, count(1), count(1)#95, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#96, avg(ss_net_paid), avg(ss_net_paid)#97) AS mergedValue#98] +Input [15]: [count(1)#83, avg(ss_ext_discount_amt)#84, avg(ss_net_paid)#85, count(1)#86, avg(ss_ext_discount_amt)#87, avg(ss_net_paid)#88, count(1)#89, avg(ss_ext_discount_amt)#90, avg(ss_net_paid)#91, count(1)#92, avg(ss_ext_discount_amt)#93, avg(ss_net_paid)#94, count(1)#95, avg(ss_ext_discount_amt)#96, avg(ss_net_paid)#97] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* Project (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * Filter (15) - +- * ColumnarToRow (14) - +- Scan parquet spark_catalog.default.store_sales (13) +Subquery:4 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] +Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(13) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] -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#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] - -(15) Filter [codegen id : 1] -Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] -Condition : ((isnotnull(ss_quantity#38) AND (ss_quantity#38 >= 21)) AND (ss_quantity#38 <= 40)) - -(16) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#39, ss_net_paid#40] -Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] - -(17) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#39, ss_net_paid#40] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#39)), partial_avg(UnscaledValue(ss_net_paid#40))] -Aggregate Attributes [5]: [count#42, sum#43, count#44, sum#45, count#46] -Results [5]: [count#47, sum#48, count#49, sum#50, count#51] - -(18) Exchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(19) HashAggregate [codegen id : 2] -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#39)), avg(UnscaledValue(ss_net_paid#40))] -Aggregate Attributes [3]: [count(1)#52, avg(UnscaledValue(ss_ext_discount_amt#39))#53, avg(UnscaledValue(ss_net_paid#40))#54] -Results [3]: [count(1)#52 AS count(1)#55, cast((avg(UnscaledValue(ss_ext_discount_amt#39))#53 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#56, cast((avg(UnscaledValue(ss_net_paid#40))#54 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#57] - -(20) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#55, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#56, avg(ss_net_paid), avg(ss_net_paid)#57) AS mergedValue#58] -Input [3]: [count(1)#55, avg(ss_ext_discount_amt)#56, avg(ss_net_paid)#57] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * Filter (23) - +- * ColumnarToRow (22) - +- Scan parquet spark_catalog.default.store_sales (21) - - -(21) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] -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#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] - -(23) Filter [codegen id : 1] -Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_quantity#59) AND (ss_quantity#59 >= 41)) AND (ss_quantity#59 <= 60)) - -(24) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#60, ss_net_paid#61] -Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] - -(25) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#60, ss_net_paid#61] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#60)), partial_avg(UnscaledValue(ss_net_paid#61))] -Aggregate Attributes [5]: [count#63, sum#64, count#65, sum#66, count#67] -Results [5]: [count#68, sum#69, count#70, sum#71, count#72] - -(26) Exchange -Input [5]: [count#68, sum#69, count#70, sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 2] -Input [5]: [count#68, sum#69, count#70, sum#71, count#72] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#60)), avg(UnscaledValue(ss_net_paid#61))] -Aggregate Attributes [3]: [count(1)#73, avg(UnscaledValue(ss_ext_discount_amt#60))#74, avg(UnscaledValue(ss_net_paid#61))#75] -Results [3]: [count(1)#73 AS count(1)#76, cast((avg(UnscaledValue(ss_ext_discount_amt#60))#74 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#77, cast((avg(UnscaledValue(ss_net_paid#61))#75 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] +Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(28) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#76, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#77, avg(ss_net_paid), avg(ss_net_paid)#78) AS mergedValue#79] -Input [3]: [count(1)#76, avg(ss_ext_discount_amt)#77, avg(ss_net_paid)#78] +Subquery:7 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet spark_catalog.default.store_sales (29) +Subquery:10 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] +Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(29) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] - -(31) Filter [codegen id : 1] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] -Condition : ((isnotnull(ss_quantity#80) AND (ss_quantity#80 >= 61)) AND (ss_quantity#80 <= 80)) - -(32) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#81, ss_net_paid#82] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] - -(33) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#81, ss_net_paid#82] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#81)), partial_avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [5]: [count#84, sum#85, count#86, sum#87, count#88] -Results [5]: [count#89, sum#90, count#91, sum#92, count#93] - -(34) Exchange -Input [5]: [count#89, sum#90, count#91, sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] - -(35) HashAggregate [codegen id : 2] -Input [5]: [count#89, sum#90, count#91, sum#92, count#93] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#81)), avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [3]: [count(1)#94, avg(UnscaledValue(ss_ext_discount_amt#81))#95, avg(UnscaledValue(ss_net_paid#82))#96] -Results [3]: [count(1)#94 AS count(1)#97, cast((avg(UnscaledValue(ss_ext_discount_amt#81))#95 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#98, cast((avg(UnscaledValue(ss_net_paid#82))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#99] - -(36) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#97, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#98, avg(ss_net_paid), avg(ss_net_paid)#99) AS mergedValue#100] -Input [3]: [count(1)#97, avg(ss_ext_discount_amt)#98, avg(ss_net_paid)#99] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* Project (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet spark_catalog.default.store_sales (37) - - -(37) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] -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#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] - -(39) Filter [codegen id : 1] -Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] -Condition : ((isnotnull(ss_quantity#101) AND (ss_quantity#101 >= 81)) AND (ss_quantity#101 <= 100)) - -(40) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#102, ss_net_paid#103] -Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] - -(41) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#102, ss_net_paid#103] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#102)), partial_avg(UnscaledValue(ss_net_paid#103))] -Aggregate Attributes [5]: [count#105, sum#106, count#107, sum#108, count#109] -Results [5]: [count#110, sum#111, count#112, sum#113, count#114] - -(42) Exchange -Input [5]: [count#110, sum#111, count#112, sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] - -(43) HashAggregate [codegen id : 2] -Input [5]: [count#110, sum#111, count#112, sum#113, count#114] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#102)), avg(UnscaledValue(ss_net_paid#103))] -Aggregate Attributes [3]: [count(1)#115, avg(UnscaledValue(ss_ext_discount_amt#102))#116, avg(UnscaledValue(ss_net_paid#103))#117] -Results [3]: [count(1)#115 AS count(1)#118, cast((avg(UnscaledValue(ss_ext_discount_amt#102))#116 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#119, cast((avg(UnscaledValue(ss_net_paid#103))#117 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#120] +Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(44) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#118, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#119, avg(ss_net_paid), avg(ss_net_paid)#120) AS mergedValue#121] -Input [3]: [count(1)#118, avg(ss_ext_discount_amt)#119, avg(ss_net_paid)#120] +Subquery:13 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] +Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] +Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] 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..18da9c02cc128 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,ss_ext_discount_amt,ss_net_paid,propagatedFilter,propagatedFilter,propagatedFilter,propagatedFilter] [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 e32db674082ee..15d841569e824 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=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] +Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#3].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket2#5, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#3].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket3#6, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#3].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket4#7, CASE WHEN (ReusedSubquery Subquery scalar-subquery#2, [id=#3].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket5#8] 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#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Output [4]: [ss_quantity#9, ss_ext_discount_amt#10, ss_net_paid#11, ss_sold_date_sk#12] 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(Or(Or(Or(Or(Or(Or(Or(Or(And(GreaterThanOrEqual(ss_quantity,1),LessThanOrEqual(ss_quantity,20)),And(GreaterThanOrEqual(ss_quantity,21),LessThanOrEqual(ss_quantity,40))),And(GreaterThanOrEqual(ss_quantity,21),LessThanOrEqual(ss_quantity,40))),And(GreaterThanOrEqual(ss_quantity,21),LessThanOrEqual(ss_quantity,40))),And(GreaterThanOrEqual(ss_quantity,41),LessThanOrEqual(ss_quantity,60))),And(GreaterThanOrEqual(ss_quantity,41),LessThanOrEqual(ss_quantity,60))),And(GreaterThanOrEqual(ss_quantity,41),LessThanOrEqual(ss_quantity,60))),And(GreaterThanOrEqual(ss_quantity,61),LessThanOrEqual(ss_quantity,80))),And(GreaterThanOrEqual(ss_quantity,61),LessThanOrEqual(ss_quantity,80))),And(GreaterThanOrEqual(ss_quantity,61),LessThanOrEqual(ss_quantity,80))),And(GreaterThanOrEqual(ss_quantity,81),LessThanOrEqual(ss_quantity,100))),And(GreaterThanOrEqual(ss_quantity,81),LessThanOrEqual(ss_quantity,100))),And(GreaterThanOrEqual(ss_quantity,81),LessThanOrEqual(ss_quantity,100)))] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Input [4]: [ss_quantity#9, ss_ext_discount_amt#10, ss_net_paid#11, ss_sold_date_sk#12] (7) Filter [codegen id : 1] -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) +Input [4]: [ss_quantity#9, ss_ext_discount_amt#10, ss_net_paid#11, ss_sold_date_sk#12] +Condition : (isnotnull(ss_quantity#9) AND ((((((((((((((ss_quantity#9 >= 1) AND (ss_quantity#9 <= 20)) OR ((ss_quantity#9 >= 21) AND (ss_quantity#9 <= 40))) OR ((ss_quantity#9 >= 21) AND (ss_quantity#9 <= 40))) OR ((ss_quantity#9 >= 21) AND (ss_quantity#9 <= 40))) OR ((ss_quantity#9 >= 41) AND (ss_quantity#9 <= 60))) OR ((ss_quantity#9 >= 41) AND (ss_quantity#9 <= 60))) OR ((ss_quantity#9 >= 41) AND (ss_quantity#9 <= 60))) OR ((ss_quantity#9 >= 61) AND (ss_quantity#9 <= 80))) OR ((ss_quantity#9 >= 61) AND (ss_quantity#9 <= 80))) OR ((ss_quantity#9 >= 61) AND (ss_quantity#9 <= 80))) OR ((ss_quantity#9 >= 81) AND (ss_quantity#9 <= 100))) OR ((ss_quantity#9 >= 81) AND (ss_quantity#9 <= 100))) OR ((ss_quantity#9 >= 81) AND (ss_quantity#9 <= 100)))) (8) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Output [7]: [ss_ext_discount_amt#10, ss_net_paid#11, (isnotnull(ss_quantity#9) AND ((ss_quantity#9 >= 1) AND (ss_quantity#9 <= 20))) AS propagatedFilter#13, (isnotnull(ss_quantity#9) AND ((ss_quantity#9 >= 21) AND (ss_quantity#9 <= 40))) AS propagatedFilter#14, (isnotnull(ss_quantity#9) AND ((ss_quantity#9 >= 41) AND (ss_quantity#9 <= 60))) AS propagatedFilter#15, (isnotnull(ss_quantity#9) AND ((ss_quantity#9 >= 61) AND (ss_quantity#9 <= 80))) AS propagatedFilter#16, (isnotnull(ss_quantity#9) AND ((ss_quantity#9 >= 81) AND (ss_quantity#9 <= 100))) AS propagatedFilter#17] +Input [4]: [ss_quantity#9, ss_ext_discount_amt#10, ss_net_paid#11, ss_sold_date_sk#12] (9) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] +Input [7]: [ss_ext_discount_amt#10, ss_net_paid#11, propagatedFilter#13, propagatedFilter#14, propagatedFilter#15, propagatedFilter#16, propagatedFilter#17] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] -Aggregate Attributes [5]: [count#21, sum#22, count#23, sum#24, count#25] -Results [5]: [count#26, sum#27, count#28, sum#29, count#30] +Functions [15]: [partial_count(1) FILTER (WHERE propagatedFilter#13), partial_avg(UnscaledValue(ss_ext_discount_amt#10)) FILTER (WHERE propagatedFilter#13), partial_avg(UnscaledValue(ss_net_paid#11)) FILTER (WHERE propagatedFilter#13), partial_count(1) FILTER (WHERE propagatedFilter#14), partial_avg(UnscaledValue(ss_ext_discount_amt#10)) FILTER (WHERE propagatedFilter#14), partial_avg(UnscaledValue(ss_net_paid#11)) FILTER (WHERE propagatedFilter#14), partial_count(1) FILTER (WHERE propagatedFilter#15), partial_avg(UnscaledValue(ss_ext_discount_amt#10)) FILTER (WHERE propagatedFilter#15), partial_avg(UnscaledValue(ss_net_paid#11)) FILTER (WHERE propagatedFilter#15), partial_count(1) FILTER (WHERE propagatedFilter#16), partial_avg(UnscaledValue(ss_ext_discount_amt#10)) FILTER (WHERE propagatedFilter#16), partial_avg(UnscaledValue(ss_net_paid#11)) FILTER (WHERE propagatedFilter#16), partial_count(1) FILTER (WHERE propagatedFilter#17), partial_avg(UnscaledValue(ss_ext_discount_amt#10)) FILTER (WHERE propagatedFilter#17), partial_avg(UnscaledValue(ss_net_paid#11)) FILTER (WHERE propagatedFilter#17)] +Aggregate Attributes [25]: [count#18, sum#19, count#20, sum#21, count#22, count#23, sum#24, count#25, sum#26, count#27, count#28, sum#29, count#30, sum#31, count#32, count#33, sum#34, count#35, sum#36, count#37, count#38, sum#39, count#40, sum#41, count#42] +Results [25]: [count#43, sum#44, count#45, sum#46, count#47, count#48, sum#49, count#50, sum#51, count#52, count#53, sum#54, count#55, sum#56, count#57, count#58, sum#59, count#60, sum#61, count#62, count#63, sum#64, count#65, sum#66, count#67] (10) Exchange -Input [5]: [count#26, sum#27, count#28, sum#29, count#30] +Input [25]: [count#43, sum#44, count#45, sum#46, count#47, count#48, sum#49, count#50, sum#51, count#52, count#53, sum#54, count#55, sum#56, count#57, count#58, sum#59, count#60, sum#61, count#62, count#63, sum#64, count#65, sum#66, count#67] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] (11) HashAggregate [codegen id : 2] -Input [5]: [count#26, sum#27, count#28, sum#29, count#30] +Input [25]: [count#43, sum#44, count#45, sum#46, count#47, count#48, sum#49, count#50, sum#51, count#52, count#53, sum#54, count#55, sum#56, count#57, count#58, sum#59, count#60, sum#61, count#62, count#63, sum#64, count#65, sum#66, count#67] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] -Aggregate Attributes [3]: [count(1)#31, avg(UnscaledValue(ss_ext_discount_amt#18))#32, avg(UnscaledValue(ss_net_paid#19))#33] -Results [3]: [count(1)#31 AS count(1)#34, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#32 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#35, cast((avg(UnscaledValue(ss_net_paid#19))#33 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#36] +Functions [15]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#10)), avg(UnscaledValue(ss_net_paid#11)), count(1), avg(UnscaledValue(ss_ext_discount_amt#10)), avg(UnscaledValue(ss_net_paid#11)), count(1), avg(UnscaledValue(ss_ext_discount_amt#10)), avg(UnscaledValue(ss_net_paid#11)), count(1), avg(UnscaledValue(ss_ext_discount_amt#10)), avg(UnscaledValue(ss_net_paid#11)), count(1), avg(UnscaledValue(ss_ext_discount_amt#10)), avg(UnscaledValue(ss_net_paid#11))] +Aggregate Attributes [15]: [count(1)#68, avg(UnscaledValue(ss_ext_discount_amt#10))#69, avg(UnscaledValue(ss_net_paid#11))#70, count(1)#71, avg(UnscaledValue(ss_ext_discount_amt#10))#72, avg(UnscaledValue(ss_net_paid#11))#73, count(1)#74, avg(UnscaledValue(ss_ext_discount_amt#10))#75, avg(UnscaledValue(ss_net_paid#11))#76, count(1)#77, avg(UnscaledValue(ss_ext_discount_amt#10))#78, avg(UnscaledValue(ss_net_paid#11))#79, count(1)#80, avg(UnscaledValue(ss_ext_discount_amt#10))#81, avg(UnscaledValue(ss_net_paid#11))#82] +Results [15]: [count(1)#68 AS count(1)#83, cast((avg(UnscaledValue(ss_ext_discount_amt#10))#69 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#84, cast((avg(UnscaledValue(ss_net_paid#11))#70 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#85, count(1)#71 AS count(1)#86, cast((avg(UnscaledValue(ss_ext_discount_amt#10))#72 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#87, cast((avg(UnscaledValue(ss_net_paid#11))#73 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#88, count(1)#74 AS count(1)#89, cast((avg(UnscaledValue(ss_ext_discount_amt#10))#75 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90, cast((avg(UnscaledValue(ss_net_paid#11))#76 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#91, count(1)#77 AS count(1)#92, cast((avg(UnscaledValue(ss_ext_discount_amt#10))#78 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#93, cast((avg(UnscaledValue(ss_net_paid#11))#79 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#94, count(1)#80 AS count(1)#95, cast((avg(UnscaledValue(ss_ext_discount_amt#10))#81 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#96, cast((avg(UnscaledValue(ss_net_paid#11))#82 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] (12) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#34, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#35, avg(ss_net_paid), avg(ss_net_paid)#36) AS mergedValue#37] -Input [3]: [count(1)#34, avg(ss_ext_discount_amt)#35, avg(ss_net_paid)#36] +Output [1]: [named_struct(count(1), count(1)#83, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#84, avg(ss_net_paid), avg(ss_net_paid)#85, count(1), count(1)#86, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#87, avg(ss_net_paid), avg(ss_net_paid)#88, count(1), count(1)#89, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#90, avg(ss_net_paid), avg(ss_net_paid)#91, 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, count(1), count(1)#95, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#96, avg(ss_net_paid), avg(ss_net_paid)#97) AS mergedValue#98] +Input [15]: [count(1)#83, avg(ss_ext_discount_amt)#84, avg(ss_net_paid)#85, count(1)#86, avg(ss_ext_discount_amt)#87, avg(ss_net_paid)#88, count(1)#89, avg(ss_ext_discount_amt)#90, avg(ss_net_paid)#91, count(1)#92, avg(ss_ext_discount_amt)#93, avg(ss_net_paid)#94, count(1)#95, avg(ss_ext_discount_amt)#96, avg(ss_net_paid)#97] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* Project (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * Filter (15) - +- * ColumnarToRow (14) - +- Scan parquet spark_catalog.default.store_sales (13) +Subquery:4 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] +Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(13) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] -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#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] - -(15) Filter [codegen id : 1] -Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] -Condition : ((isnotnull(ss_quantity#38) AND (ss_quantity#38 >= 21)) AND (ss_quantity#38 <= 40)) - -(16) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#39, ss_net_paid#40] -Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] - -(17) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#39, ss_net_paid#40] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#39)), partial_avg(UnscaledValue(ss_net_paid#40))] -Aggregate Attributes [5]: [count#42, sum#43, count#44, sum#45, count#46] -Results [5]: [count#47, sum#48, count#49, sum#50, count#51] - -(18) Exchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(19) HashAggregate [codegen id : 2] -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#39)), avg(UnscaledValue(ss_net_paid#40))] -Aggregate Attributes [3]: [count(1)#52, avg(UnscaledValue(ss_ext_discount_amt#39))#53, avg(UnscaledValue(ss_net_paid#40))#54] -Results [3]: [count(1)#52 AS count(1)#55, cast((avg(UnscaledValue(ss_ext_discount_amt#39))#53 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#56, cast((avg(UnscaledValue(ss_net_paid#40))#54 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#57] - -(20) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#55, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#56, avg(ss_net_paid), avg(ss_net_paid)#57) AS mergedValue#58] -Input [3]: [count(1)#55, avg(ss_ext_discount_amt)#56, avg(ss_net_paid)#57] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * Filter (23) - +- * ColumnarToRow (22) - +- Scan parquet spark_catalog.default.store_sales (21) - - -(21) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] -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#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] - -(23) Filter [codegen id : 1] -Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_quantity#59) AND (ss_quantity#59 >= 41)) AND (ss_quantity#59 <= 60)) - -(24) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#60, ss_net_paid#61] -Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] - -(25) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#60, ss_net_paid#61] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#60)), partial_avg(UnscaledValue(ss_net_paid#61))] -Aggregate Attributes [5]: [count#63, sum#64, count#65, sum#66, count#67] -Results [5]: [count#68, sum#69, count#70, sum#71, count#72] - -(26) Exchange -Input [5]: [count#68, sum#69, count#70, sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 2] -Input [5]: [count#68, sum#69, count#70, sum#71, count#72] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#60)), avg(UnscaledValue(ss_net_paid#61))] -Aggregate Attributes [3]: [count(1)#73, avg(UnscaledValue(ss_ext_discount_amt#60))#74, avg(UnscaledValue(ss_net_paid#61))#75] -Results [3]: [count(1)#73 AS count(1)#76, cast((avg(UnscaledValue(ss_ext_discount_amt#60))#74 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#77, cast((avg(UnscaledValue(ss_net_paid#61))#75 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] +Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(28) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#76, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#77, avg(ss_net_paid), avg(ss_net_paid)#78) AS mergedValue#79] -Input [3]: [count(1)#76, avg(ss_ext_discount_amt)#77, avg(ss_net_paid)#78] +Subquery:7 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet spark_catalog.default.store_sales (29) +Subquery:10 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] +Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(29) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] - -(31) Filter [codegen id : 1] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] -Condition : ((isnotnull(ss_quantity#80) AND (ss_quantity#80 >= 61)) AND (ss_quantity#80 <= 80)) - -(32) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#81, ss_net_paid#82] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] - -(33) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#81, ss_net_paid#82] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#81)), partial_avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [5]: [count#84, sum#85, count#86, sum#87, count#88] -Results [5]: [count#89, sum#90, count#91, sum#92, count#93] - -(34) Exchange -Input [5]: [count#89, sum#90, count#91, sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] - -(35) HashAggregate [codegen id : 2] -Input [5]: [count#89, sum#90, count#91, sum#92, count#93] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#81)), avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [3]: [count(1)#94, avg(UnscaledValue(ss_ext_discount_amt#81))#95, avg(UnscaledValue(ss_net_paid#82))#96] -Results [3]: [count(1)#94 AS count(1)#97, cast((avg(UnscaledValue(ss_ext_discount_amt#81))#95 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#98, cast((avg(UnscaledValue(ss_net_paid#82))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#99] - -(36) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#97, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#98, avg(ss_net_paid), avg(ss_net_paid)#99) AS mergedValue#100] -Input [3]: [count(1)#97, avg(ss_ext_discount_amt)#98, avg(ss_net_paid)#99] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* Project (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet spark_catalog.default.store_sales (37) - - -(37) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] -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#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] - -(39) Filter [codegen id : 1] -Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] -Condition : ((isnotnull(ss_quantity#101) AND (ss_quantity#101 >= 81)) AND (ss_quantity#101 <= 100)) - -(40) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#102, ss_net_paid#103] -Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] - -(41) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#102, ss_net_paid#103] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#102)), partial_avg(UnscaledValue(ss_net_paid#103))] -Aggregate Attributes [5]: [count#105, sum#106, count#107, sum#108, count#109] -Results [5]: [count#110, sum#111, count#112, sum#113, count#114] - -(42) Exchange -Input [5]: [count#110, sum#111, count#112, sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] - -(43) HashAggregate [codegen id : 2] -Input [5]: [count#110, sum#111, count#112, sum#113, count#114] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#102)), avg(UnscaledValue(ss_net_paid#103))] -Aggregate Attributes [3]: [count(1)#115, avg(UnscaledValue(ss_ext_discount_amt#102))#116, avg(UnscaledValue(ss_net_paid#103))#117] -Results [3]: [count(1)#115 AS count(1)#118, cast((avg(UnscaledValue(ss_ext_discount_amt#102))#116 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#119, cast((avg(UnscaledValue(ss_net_paid#103))#117 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#120] +Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -(44) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#118, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#119, avg(ss_net_paid), avg(ss_net_paid)#120) AS mergedValue#121] -Input [3]: [count(1)#118, avg(ss_ext_discount_amt)#119, avg(ss_net_paid)#120] +Subquery:13 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] +Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] +Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] 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..18da9c02cc128 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,ss_ext_discount_amt,ss_net_paid,propagatedFilter,propagatedFilter,propagatedFilter,propagatedFilter] [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/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 68f14f13bbd66..66bbe3ee90056 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2383,6 +2383,35 @@ class SubquerySuite extends QueryTest } } + test("Merge non-correlated scalar subqueries with different filters") { + withSQLConf(SQLConf.PLAN_MERGE_FILTER_PROPAGATION_ENABLED.key -> "true") { + Seq(false, true).foreach { enableAQE => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString) { + val df = sql( + """ + |SELECT + | (SELECT max(key) AS key FROM testData WHERE value < '50'), + | (SELECT sum(key) AS key FROM testData WHERE value = '50'), + | (SELECT count(distinct key) AS key FROM testData WHERE value > '50') + """.stripMargin) + + checkAnswer(df, Row(100, 50, 53) :: Nil) + + val plan = df.queryExecution.executedPlan + val subqueryIds = collectWithSubqueries(plan) { case s: SubqueryExec => s.id } + val reusedSubqueryIds = collectWithSubqueries(plan) { + case rs: ReusedSubqueryExec => rs.child.id + } + + assert(subqueryIds.size == 1, "Missing or unexpected SubqueryExec in the plan") + assert(reusedSubqueryIds.size == 2, + "Missing or unexpected reused ReusedSubqueryExec in the plan") + } + } + } + } + test("SPARK-39355: Single column uses quoted to construct UnresolvedAttribute") { checkAnswer( sql("""