diff --git a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ConstraintSetImplicit.scala b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ConstraintSetImplicit.scala new file mode 100644 index 000000000000..991c4513806e --- /dev/null +++ b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ConstraintSetImplicit.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import scala.collection.mutable +import scala.language.implicitConversions + +object ConstraintSetImplicit { + implicit def toImplicitWrapper[T](self: scala.collection.Iterable[T]): Wrapper[T] = + new Wrapper(self) + + class Wrapper[T](val coll: scala.collection.Iterable[T]) { + + def toMutableSet(mutableSet: mutable.Set.type): mutable.Set[T] = coll.to[mutable.Set] + + def toMutableBuffer(mutableBuffer: mutable.Buffer.type): mutable.Buffer[T] = + coll.to[mutable.Buffer] + + } +} + diff --git a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ExpressionMap.scala b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ExpressionMap.scala new file mode 100644 index 000000000000..1daad83f57e3 --- /dev/null +++ b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ExpressionMap.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import scala.collection.mutable + +object ExpressionMap { + /** Constructs a new [[ExpressionMap]] by applying [[Canonicalize]] to `expressions`. */ + def apply[T](map: mutable.Map[Expression, T]): ExpressionMap[T] = { + val newMap = new ExpressionMap[T](map) + map.foreach(newMap.add) + newMap + } +} + +/** + * A helper class created on the lines of [[AttributeMap]] and [[ExpressionSet]] + * The key added in the Map is always stored in canonicalized form. + * + * @param baseMap The underlying Map object which is either empty or pre-populated with all the + * keys being canonicalized form of expressions + * @tparam T The value part of the Map + */ +class ExpressionMap[T](val baseMap: mutable.Map[Expression, T] = new mutable.HashMap[Expression, T]) + extends mutable.Map[Expression, T] { + + override def get(expr: Expression): Option[T] = + baseMap.get(expr.canonicalized) + + + protected def add(tup: (Expression, T)): Unit = { + + this.baseMap += (tup._1.canonicalized -> tup._2) + + } + + override def contains(expr: Expression): Boolean = + baseMap.contains(expr.canonicalized) + + override def +=(kv: (Expression, T)): this.type = { + this.add(kv) + this + } + + override def iterator: Iterator[(Expression, T)] = baseMap.iterator + + override def -=(key: Expression): this.type = { + this.baseMap -= key.canonicalized + this + } +} diff --git a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/ConstraintSetImplicit.scala b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/ConstraintSetImplicit.scala new file mode 100644 index 000000000000..cbd73b05ad24 --- /dev/null +++ b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/ConstraintSetImplicit.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import scala.collection.mutable +import scala.language.implicitConversions + +object ConstraintSetImplicit { + implicit def toImplicitWrapper[T](self: scala.collection.Iterable[T]): Wrapper[T] = + new Wrapper(self) + + class Wrapper[T](val coll: scala.collection.Iterable[T]) { + + def toMutableSet(mutableSet: mutable.Set.type): mutable.Set[T] = + coll.to(mutable.Set) + + def toMutableBuffer(mutableBuffer: mutable.Buffer.type): mutable.Buffer[T] = + coll.to(mutable.Buffer) + } +} + diff --git a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/ExpressionMap.scala b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/ExpressionMap.scala new file mode 100644 index 000000000000..747b4256f2cb --- /dev/null +++ b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/ExpressionMap.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import scala.collection.mutable + +object ExpressionMap { + /** Constructs a new [[ExpressionMap]] by applying [[Canonicalize]] to `expressions`. */ + def apply[T](map: mutable.Map[Expression, T]): ExpressionMap[T] = { + val newMap = new ExpressionMap[T](map) + map.foreach(newMap.addOne) + newMap + } +} + +/** + * A helper class created on the lines of [[AttributeMap]] and [[ExpressionSet]] + * The key added in the Map is always stored in canonicalized form. + * + * @param baseMap The underlying Map object which is either empty or pre-populated with all the + * keys being canonicalized form of expressions + * @tparam T The value part of the Map + */ +class ExpressionMap[T](val baseMap: mutable.Map[Expression, T] = new mutable.HashMap[Expression, T]) + extends mutable.Map[Expression, T] { + + override def get(expr: Expression): Option[T] = baseMap.get(expr.canonicalized) + + + override def addOne(tup: (Expression, T)): this.type = { + this.baseMap += (tup._1.canonicalized -> tup._2) + this + } + + override def contains(expr: Expression): Boolean = baseMap.contains(expr.canonicalized) + + override def iterator: Iterator[(Expression, T)] = baseMap.iterator + + override def subtractOne(key: Expression): this.type = { + this.baseMap -= key.canonicalized + this + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ConstraintSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ConstraintSet.scala new file mode 100644 index 000000000000..7d8fbae5a716 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ConstraintSet.scala @@ -0,0 +1,826 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import scala.collection.{mutable, GenTraversableOnce} +import scala.collection.mutable.ArrayBuffer + +import ConstraintSetImplicit._ + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +/** + * This class stores the constraints available at each node. + * The constraint expressions are stored in canonicalized form. + * The major way in which it differs from the [[ExpressionSet]] is that + * in case of Project Node, it stores information about the aliases + * and groups them on the basis of equivalence. In stock spark all the + * constraints are pre-created pessimistically for all possible combinations + * of equivalent aliases. While in this class only one constraint per filter + * is present & the rest are created in expand function at the time of + * new filter inference. Also even then, not all combinations are created, + * instead one filter constraint for each alias in the group is created. + * + * The core logic of new algorithm is as follows + * The base constraints present in this object will always be composed of + * as far as possible, of those attributes which were present in the incoming + * set & are also part of the output set for the node. If any attribute or expression + * is part of any outgoing alias, they will be added to either attribute equivalence list + * or expression equivalence list, depending upon whether the Alias's child is attribute + * or generic expression. The 0th element of each of the buffer in the attribute equivalence + * list & expression equivalence list are special in the sense, that any constraint, + * if it is referring to any attribute or expression in the two lists, is guaranteed to + * use the 0th element and not any other members. An attempt is made to ensure that the + * constraint survives when bubbling up. If an attribute or expression, which is part + * of incoming constraint, but is not present in output set, makes the survival of the + * constraint susceptible. In such case, the attribute equivalence list & expression + * equivalence list are consulted & if found that the buffer containing 0th element as + * the attribute which is getting removed, has another element, then that element ( the + * 1th member) is chosen to replace the attribute being removed in the constraint. + * The constraint is updated to use the 1th element. This 1th element is then put in + * 0th place of the buffer. + * It is to be noted that attribute equivalence list will have buffers where each + * element will be of type attribute only. While expression equivalence list will + * have 0th element as of type generic expression, rest being attributes. + * It is also to be noted, that the 0th element of expression equivalence list being + * generic expression, itself is composed of attributes. And the expression equivalence + * list needs to be updated, if any of the attribute it refers to is being eliminated + * from the output set. + * + * For eg. consider an existing constraint a + b + c + d > 7 + * let the input set comprise of attributes a, b, c, d + * Let the output set be a, a as a1, a as a2, b + c as z, d as d1 + * In the above d , b & c are getting eliminated + * while a survives and also has a1 & a2. + * d is referred as d1. + * the initial attribute equivalence list will be + * a, a1, a2 + * d, d1 + * expression equivalence list will be + * b + c , z + * Now for the constraint a + b + c + d > 7 to survive + * b + c => can be replaced by z + * d can be replaced by d1 + * so the constraint will be updated as + * a + z + d1 > 7 + * the updated attribute equivalence list will be + * a, a1, a2 + * since d1 will be left alone, it will no longer be part of the list + * same is the case with expression equivalence list. + * as b + c, will be removed, only z1 remains, so it will be removed + * from expression equivalence list & it will be empty. + * + * @param baseSet [[mutable.Set[Expression]] which contains Canonicalized Constraint Expression + * @param originals [[mutable.Buffer[Expression]] buffer containing the original constraint + * expression + * @param attribRefBasedEquivalenceList A List of List which contains grouping of equivalent + * Aliases referring to same Attribute + * @param expressionBasedEquivalenceList A List of List which contains grouping of equivalent + * Aliases referring to same Expression( which is not an Attribute) + */ + +class ConstraintSet private( + baseSet: mutable.Set[Expression], + originals: mutable.Buffer[Expression] = new ArrayBuffer, + val attribRefBasedEquivalenceList: Seq[mutable.Buffer[Expression]], + val expressionBasedEquivalenceList: Seq[mutable.Buffer[Expression]] +) extends ExpressionSet(baseSet, originals) with Logging { + + import ConstraintSetImplicit._ + def this(actuals: mutable.Buffer[Expression]) = + this (actuals.map(_.canonicalized).toMutableSet(mutable.Set), actuals, + Seq.empty[mutable.Buffer[Expression]], Seq.empty[mutable.Buffer[Expression]]) + + def this( + actuals: mutable.Buffer[Expression], + attribRefBasedEquivalenceList: Seq[mutable.Buffer[Expression]], + expressionBasedEquivalenceList: Seq[mutable.Buffer[Expression]]) = + this(actuals.map(_.canonicalized).toMutableSet(mutable.Set), actuals, + attribRefBasedEquivalenceList, expressionBasedEquivalenceList) + + def this(baseSet: mutable.Set[Expression], actuals: mutable.Buffer[Expression]) = + this(baseSet, actuals, Seq.empty[mutable.Buffer[Expression]], + Seq.empty[mutable.Buffer[Expression]]) + + def this() = this(mutable.Buffer.empty[Expression]) + + override def clone(): ConstraintSet = new ConstraintSet(baseSet.clone(), + originals.clone(), this.attribRefBasedEquivalenceList.map(_.clone()), + this.expressionBasedEquivalenceList.map(_.clone())) + + override def union(that: ExpressionSet): ExpressionSet = { + def unionEquivList(thisList: Seq[mutable.Buffer[Expression]], + thatList: Seq[mutable.Buffer[Expression]]): Seq[mutable.Buffer[Expression]] + = { + val zerothElems = thisList.map(_.head) + val (common, other) = thatList.partition(buff => zerothElems. + exists(buff.head.canonicalized == _.canonicalized)) + val copy = thisList.map(_.clone()) + common.foreach(commonBuff => { + val copyBuff = copy.find(_.head.canonicalized == commonBuff.head.canonicalized).get + commonBuff.drop(1).foreach(expr => if (!copyBuff.exists( + _.canonicalized == expr.canonicalized)) { + copyBuff += expr + }) + }) + copy ++ other.map(_.clone()) + } + + def removeAnyResidualDuplicateAttribute(newAttribList: Seq[mutable.Buffer[Expression]]): + (Seq[mutable.Buffer[Expression]], Boolean) + = { + val allAttribs = newAttribList.flatten + var foundEmptyBuff = false + var foundDuplicates = false + allAttribs.foreach(attrib => { + val buffs = newAttribList.filter(buff => buff.exists(_.canonicalized == + attrib.canonicalized)) + if (buffs.size > 1) { + foundDuplicates = true + val rests = buffs.drop(1) + rests.foreach(buff => { + ConstraintSet.removeCanonicalizedExpressionFromBuffer(buff, attrib) + if (buff.isEmpty) { + foundEmptyBuff = true + } + }) + } + }) + (if (foundEmptyBuff) { + newAttribList.filterNot(_.isEmpty) + } else { + newAttribList + }) -> foundDuplicates + } + + val (newAttribList, newExpEquivList) = that match { + case thatX: ConstraintSet => + (unionEquivList(this.attribRefBasedEquivalenceList, thatX.attribRefBasedEquivalenceList), + unionEquivList(this.expressionBasedEquivalenceList, thatX.expressionBasedEquivalenceList)) + case _ => (this.attribRefBasedEquivalenceList.map(_.clone()), + this.expressionBasedEquivalenceList.map(_.clone())) + } + val (cleanedAttribList, foundDuplicates) = removeAnyResidualDuplicateAttribute(newAttribList) + if (foundDuplicates) { + val errorMessage = s"Found same attribute ref present in more than 1 buffers." + + s"This indicates either a faulty plan involving same dataframe reference self joined" + + s"without alias or something murkier" + throwExceptionOrLogWarning(errorMessage) + } + val newSet = new ConstraintSet(this.baseSet.clone(), this.originals.clone(), cleanedAttribList, + newExpEquivList) + ConstraintSet.addFiltersToConstraintSet(that, newSet) + newSet + } + + override def constructNew(newBaseSet: mutable.Set[Expression] = new mutable.HashSet, + newOriginals: mutable.Buffer[Expression] = new ArrayBuffer): ExpressionSet = { + new ConstraintSet(newBaseSet, newOriginals, this.attribRefBasedEquivalenceList.map(_.clone()), + this.expressionBasedEquivalenceList.map(_.clone())) + } + + /** + * Converts the given expression to the canonicalized form, needed for ConstraintSet + * For eg lets assume expression equivalence list contains a buffer with following + * entries. Below a & b are attributes of the output & z is an alias of a + b + * a + b, z + * If the expression to be added to constraints is z > 10, then it should be + * entered in the constraintset in terms of primary attributes (i.e a + b) + * so on canonicalization z > 10 will be converted to a + b > 10 + * @param ele Expression to be canonicalized + * @return Expression which is canonicalized + */ + override def convertToCanonicalizedIfRequired(ele: Expression): Expression = { + if (this.baseSet.contains(ele.canonicalized)) { + ele + } else { + val suspectAttribs = ele.references.toSet -- + this.attribRefBasedEquivalenceList.map(buff => buff.head.asInstanceOf[Attribute]) + if (suspectAttribs.isEmpty) { + ele + } else { + val mappings = suspectAttribs.map(attrib => + (this.attribRefBasedEquivalenceList ++ this.expressionBasedEquivalenceList).find( + buff => buff.exists(_.canonicalized == attrib.canonicalized)). + map(buff => attrib -> Option(buff.head)).getOrElse(attrib -> None)).toMap + ele.transformUp { + case attr: Attribute if mappings.contains(attr) => mappings(attr).getOrElse(attr) + } + } + } + } + + + /** + * This function updates the existing non redundant, non trivial constraints stored + * as per the basis of incoming attributes and outgoing attributes of the node. + * If the attributes forming the constraints are not going to be part of the output set, + * then attempt is made ,as much as possible, to see if the constraint can survive + * by modifying it with 1st available alias for the attribute getting removed. + * It also tracks the aliases of the attribute which are then used to generate + * redundant constraints in the expand function & for pruning in the contains function + * + * @param outputAttribs The attributes which are part of the output set + * @param inputAttribs The attributes which make up the incoming attributes + * @param projectList The list of projections containing the NamedExpression + * @param oldAliasedConstraintsCreator A partial function used for generating all + * combination of constraints as per old code. + * Used only when the un optimized constraint propagation + * is used. Used in ExpressionSet + * @return The new valid ConstraintSet + */ + override def updateConstraints(outputAttribs: Seq[Attribute], + inputAttribs: Seq[Attribute], projectList: Seq[NamedExpression], + oldAliasedConstraintsCreator: Option[Seq[NamedExpression] => ExpressionSet]): + ConstraintSet = { + val (aliasBased, _) = projectList.partition { + case _: Alias => true + case _ => false + } + val groupHeadToGroupMap: ExpressionMap[mutable.Buffer[Expression]] = + new ExpressionMap[mutable.Buffer[Expression]]() + + this.attribRefBasedEquivalenceList.foreach(x => + groupHeadToGroupMap += (x.head -> x.clone()) + ) + this.expressionBasedEquivalenceList.foreach(x => + groupHeadToGroupMap += (x.head -> x.clone()) + ) + // clone the keys so that the set obtained is static & detached from + // the groupHeadToGroupMap + val existingAttribGroupHead = groupHeadToGroupMap.keySet.toSet + + // add the incoming attribs list + aliasBased.foreach( ne => ne match { + case al: Alias => + // find the group to which this alias's child belongs to + // if the child is an attribute + val alChild = al.child + val key = this.attribRefBasedEquivalenceList + .find(_.exists(_.canonicalized == alChild.canonicalized)).map(_.head) + .getOrElse(this.expressionBasedEquivalenceList.find(buff => + buff.exists(_.canonicalized == alChild.canonicalized)).map(_.head).getOrElse(alChild)) + + groupHeadToGroupMap.get(key) match { + case Some(seq) => seq += al.toAttribute + case None => + // if key is a literal then it is observed that the Alias may have + // an exprID which is somehow getting repeated across multiple nodes. + // However the code below checks for presence of the alias's exprID + // irrespective of the Alias's child is Literal or any general Expression. + // (At this point cannot completely rule out repetition of non Literals Alias) + // In such case even if the key is not in groupHead , the exprId might still be + // in attribute reference list. So before creating a new entry in the map + // check if the exprId of the alias is present in the attribute equivalence list. + // If it is present already, skip its entry + if (!this.attribRefBasedEquivalenceList.exists(buff => buff.exists(_.canonicalized == + al.toAttribute.canonicalized))) { + val temp: mutable.Buffer[Expression] = mutable.ArrayBuffer(al.child, al.toAttribute) + groupHeadToGroupMap += al.child -> temp + } + } + case _ => // not expected + }) + + + // Find those incoming attributes which are not projecting out + val attribsRemoved = inputAttribs.filterNot(attr => + outputAttribs.exists(_.canonicalized == attr.canonicalized)).toSet + // for each of the attribute getting removed , find replacement if any + val replaceableAttributeMap: ExpressionMap[Attribute] = new ExpressionMap[Attribute]() + fillReplacementOrClearGroupHeadForRemovedAttributes(attribsRemoved, replaceableAttributeMap, + groupHeadToGroupMap) + val (attribBasedEquivalenceList, initialExprBasedEquivalenceList) = { + val tup = groupHeadToGroupMap.values.partition(buff => + buff.head match { + case _: Attribute => true + case _ => false + } + ) + (tup._1.toMutableBuffer(mutable.Buffer), + tup._2.toMutableBuffer(mutable.Buffer)) + } + + // now work on expression (other than attribute based) + val replaceableExpressionMap: ExpressionMap[Attribute] = new ExpressionMap[Attribute]() + val exprBasedEquivalenceList = getUpdatedExpressionEquivalenceListWithSideEffects( + attribsRemoved, replaceableAttributeMap, replaceableExpressionMap, + attribBasedEquivalenceList, initialExprBasedEquivalenceList) + + // Now update or remove the filters depending upon which + // can survive based on replacement available + val updatedFilterExprs = getUpdatedConstraints(attribsRemoved, replaceableAttributeMap, + replaceableExpressionMap) + + exprBasedEquivalenceList.foreach(buffer => { + val expr = buffer.head + if (!existingAttribGroupHead.contains(expr.canonicalized)) { + val newConstraintOpt = if (expr.references.isEmpty) { + buffer.remove(0) + expr match { + case NonNullLiteral(_, _) | _: NullIntolerant => Some(EqualTo(buffer.head, expr)) + case _ => Some(EqualNullSafe(buffer.head, expr)) + } + } else { + None + } + newConstraintOpt.foreach(newConstraint => if (!updatedFilterExprs.exists(_.canonicalized == + newConstraint.canonicalized)) { + updatedFilterExprs += newConstraint + }) + } + }) + + // remove all mappings of constants as the expression + // also identify those elements which are plain attribute based only + // so that we transfer them to attribute ref list + val (attribsOnly, newExprBasedEquivalenceList) = exprBasedEquivalenceList.filter(buff => + buff.head.references.nonEmpty && buff.size > 1).partition(_.head match { + case _: Attribute => true + case _ => false + }) + // Now filter the attribBasedEquivalenceList which only has 1 element + // This is because if there is only 1 element in the buffer, it cannot + // be of any help in making a constraint survive, in case that attribute + // is not part of output set, so no point in keeping it in the attribute + // equivalence list. + val newAttribBasedEquivalenceList = (attribBasedEquivalenceList ++ attribsOnly). + filter(_.size > 1) + val canonicalized = updatedFilterExprs.map(_.canonicalized).toMutableSet(mutable.Set) + + if (canonicalized.size != updatedFilterExprs.size) { + this.logError(s"ConstraintSet::updateConstraints: Canonicalized filter" + + s"expression set not matching with updated filter expressions, indicating duplicate" + + s" filters.") + val duplicateFilters = mutable.ArrayBuffer[Seq[Expression]]() + canonicalized.foreach(canon => { + val tempExprs = updatedFilterExprs.filter(_.canonicalized == canon).toSeq + if (tempExprs.size > 1) { + duplicateFilters += tempExprs + } + }) + if (Utils.isTesting) { + val errorMessage = s"Found following duplicate filters" + + s" ${duplicateFilters.flatten.mkString(",")}" + assert(false, errorMessage) + } else { + duplicateFilters.foreach(duplicates => { + logError(s"ConstraintSet::updateConstraints: duplicate filters" + + s" = ${duplicates.mkString("::")}") + duplicates.drop(1).foreach(x => { + val indx = updatedFilterExprs.indexWhere(_ == x) + if (indx != -1) { + updatedFilterExprs.remove(indx) + } + }) + }) + } + } + + + new ConstraintSet(canonicalized, updatedFilterExprs, newAttribBasedEquivalenceList.toSeq, + newExprBasedEquivalenceList.toSeq) + } + + private def getUpdatedExpressionEquivalenceListWithSideEffects(attribsRemoved: Set[Attribute], + replaceableAttributeMap: ExpressionMap[Attribute], + replaceableExpressionMap: ExpressionMap[Attribute], + attribBasedEquivalenceList: mutable.Buffer[mutable.Buffer[Expression]], + initialExprBasedEquivalenceList: mutable.Buffer[mutable.Buffer[Expression]] + ): mutable.Buffer[mutable.Buffer[Expression]] = { + initialExprBasedEquivalenceList.map(buff => { + val zerothElem = buff.head + val refs = zerothElem.references + if (refs.nonEmpty) { + if (refs.exists(ref => attribsRemoved.exists(_.canonicalized == ref.canonicalized))) { + val newZeroth = zerothElem.transformUp { + case attr: Attribute => + if (attribsRemoved.exists( _.canonicalized == attr.canonicalized)) { + replaceableAttributeMap.get(attr) match { + case Some(x) => x + case None => attr + } + } else attr + } + if (newZeroth.references.exists(ref => + attribsRemoved.exists(_.canonicalized == ref.canonicalized))) { + val removedExpression = buff.remove(0) + if (buff.nonEmpty) { + replaceableExpressionMap += (removedExpression -> buff.head + .asInstanceOf[Attribute]) + } + // If the buffer size after removal is > 1 + // transfer the remaining attributes in the buffer to attrib equivalent list + // If the buffer size == 1, then it will be removed in the final filtration + // as the buffer size == 1 implies that the 0th position expression cannot + // survive up the chain, if any of the attribute it is referencing is lost, + // as there is no alias to support it + if (buff.size > 1) { + assert(buff.forall { + case _: Attribute => true + case _ => false + }) + if (buff.nonEmpty) { + val preexistingExprs = buff.filter(expr => attribBasedEquivalenceList.exists( + buffx => buffx.exists(_.canonicalized == expr.canonicalized))) + preexistingExprs.foreach(expr => { + val index = buff.indexWhere(_.canonicalized == expr.canonicalized) + buff.remove(index) + }) + if (buff.nonEmpty) { + attribBasedEquivalenceList += buff + } + } + mutable.Buffer.empty[Expression] + } else { + buff + } + } else { + buff(0) = newZeroth + buff + } + } else { + buff + } + } else { + attribsRemoved.foreach(ConstraintSet.removeCanonicalizedExpressionFromBuffer(buff, _)) + buff + } + }).filter(_.size > 1) + // The above filtering ensures that if the buffer size after removal is 1, + // then purge the buffer + } + + private def getUpdatedConstraints(attribsRemoved: Set[Attribute], + replaceableAttributeMap: ExpressionMap[Attribute], + replaceableExpressionMap: ExpressionMap[Attribute]): mutable.Buffer[Expression] = { + this.originals.flatMap(filterExpr => { + val attribRefs = filterExpr.references + if (attribRefs.isEmpty) { + Set.empty[Expression] + } else { + if (attribRefs.exists(ref => attribsRemoved.exists(_.canonicalized == ref.canonicalized))) { + val newFilterExp = filterExpr.transformUp { + case attr: Attribute => + if (attribsRemoved.exists(_.canonicalized == attr.canonicalized)) { + replaceableAttributeMap.get(attr) match { + case Some(x) => x + case None => attr + } + } else attr + } + // if filter still contains attribs which will be removed, + // below code checks if filter can survive by replacement with a complex expression + if (newFilterExp.references.exists(ref => + attribsRemoved.exists(_.canonicalized == ref.canonicalized))) { + val newNewFilterExp = newFilterExp.transformUp { + case expr: Expression => + replaceableExpressionMap.get(expr) match { + case Some(x) => x + case None => expr + } + } + if (newNewFilterExp.references.exists(ref => + attribsRemoved.exists(_.canonicalized == ref.canonicalized))) { + Set.empty[Expression] + } else { + Set(newNewFilterExp).filterNot(x => + this.originals.exists(_.canonicalized == x.canonicalized)) + } + } else { + Set(newFilterExp).filterNot(x => + this.originals.exists(_.canonicalized == x.canonicalized)) + } + } else { + Set(filterExpr) + } + } + }) + } + + private def fillReplacementOrClearGroupHeadForRemovedAttributes(attribsRemoved: Set[Attribute], + replaceableAttributeMap: ExpressionMap[Attribute], + groupHeadToGroupMap: ExpressionMap[mutable.Buffer[Expression]]): Unit = { + attribsRemoved.foreach(attrib => { + groupHeadToGroupMap.get(attrib) match { + case Some(buff) => + if (attrib.canonicalized == buff.head.canonicalized) { + buff.remove(0) + } else { + val errorMessage = s"ConstraintSet::fillReplacementOrClearGroupHead..:PRISM-77994:" + + s"GroupHead =$attrib not matching with the buffer head = ${buff.head}." + + s"Not removing the head from the buffer" + if (Utils.isTesting) { + assert(false, errorMessage) + } else { + this.logError(errorMessage) + } + } + // remove any attributes which may be in position other than 0 in the buffer + attribsRemoved.foreach(x => ConstraintSet.removeCanonicalizedExpressionFromBuffer(buff, + x)) + // if there is no replacement and the attribute being removed + // was the only one present, then the buffer is purged + // else replaced by updated key + // groupHeadToGroupMap.remove(exprRemoved) + groupHeadToGroupMap.remove(attrib) + if (buff.nonEmpty) { + replaceableAttributeMap += attrib -> buff.head.asInstanceOf[Attribute] + groupHeadToGroupMap.put(buff.head.asInstanceOf[Attribute], buff) + } + case None => // there may be attributes which are removed but lying in + // position other than 0th + // which may be such that zeroth attrib is not present in the list of attrib being + // removed & hence escaped in above op. so we need to again filter the map + // at this point it is guaranteed that once the filtering has happened , there will + // be no buffer which can be empty + val errorKeys = mutable.ArrayBuffer[Expression]() + groupHeadToGroupMap.foreach { + case (key, buffer) => val initialHead = buffer.head + // The operation below should not touch the head of any buffer + ConstraintSet.removeCanonicalizedExpressionFromBuffer(buffer, attrib) + // To debug PRISM-77994 disabling the assert instead and logging & recovering + // assert(buffer.nonEmpty) + if (buffer.isEmpty || initialHead != buffer.head) { + errorKeys += key + this.logError(s"ConstraintSet::fillReplacementOrClearGroupHead:PRISM-77994:" + + s"for non GroupHead key attribute $attrib, It still modified the 0th" + + s" position of buffer with group head key $key." + + s" The initial head of buffer was $initialHead") + } + } + if (errorKeys.nonEmpty) { + if (Utils.isTesting) { + val errorMessage = s"ConstraintSet::fillReplacementOrClearGroupHead:PRISM-77994:" + + s"for non GroupHead key attribute $attrib, found following modified grouphead" + + s" keys. ${errorKeys.mkString(",")}" + assert(false, errorMessage) + } else { + errorKeys.foreach(key => { + val oldValOpt = groupHeadToGroupMap.remove(key) + oldValOpt.foreach(buff => if (buff.nonEmpty) { + groupHeadToGroupMap.put(buff.head, buff) + }) + }) + } + } + } + }) + } + + override def withNewConstraints(filters: ExpressionSet): ConstraintSet = { + val newConstraintSet = new ConstraintSet(mutable.Buffer[Expression](), + this.attribRefBasedEquivalenceList.map(_.clone()), + this.expressionBasedEquivalenceList.map(_.clone)) + ConstraintSet.addFiltersToConstraintSet(filters, newConstraintSet) + newConstraintSet + } + + + override def attributesRewrite(mapping: AttributeMap[Attribute]): ConstraintSet = { + val transformer: PartialFunction[Expression, Expression] = { + case a: Attribute => mapping(a) + } + val newOriginals = this.originals.map(x => x.transformUp(transformer)) + val newAttribBasedEquiList = this.attribRefBasedEquivalenceList.map(buff => + buff.map(x => x.transformUp(transformer))) + val newExpBasedEquiList = this.expressionBasedEquivalenceList.map(buff => + buff.map(x => x.transformUp(transformer))) + val newConstraintSet = new ConstraintSet(mutable.Buffer[Expression](), + newAttribBasedEquiList, newExpBasedEquiList) + ConstraintSet.addFiltersToConstraintSet(newOriginals, newConstraintSet) + newConstraintSet + } + + /** + * This function is used during pruning and also when any new condition is being + * added to the constraintset. The idea is that existing conditions in the constraintset + * are the bare minimum essential (non redundant) constraints. So any filter to be checked + * if it can be pruned or not can be checked using this function, if that filter is + * derivable using the constraints available. If it is derivable it means the filter is + * redundant and can be pruned. Also if any new constraint is being added to the + * constraintset that also can be checked if it is redundant or not. If redundant, + * it will not get added. This method converts the incoming expression into its + * constituents attributes before being checked. + * For. eg if the incoming expression is say z + c > 10, where z is an alias of base + * attributes a + b. And say constraintset already contains a condition + * a + b + c > 10. Then z + c > 10, is converted into a + b + c > 10 making use + * of tacking data of aliases, and it will be found in the constraintset & + * contains will return as true + * @param elem Expression to be checked if it is redundant or not. + * @return boolean true if it already exists in constraintset( is redundant) + */ + override def contains(elem: Expression): Boolean = { + if (super.contains(elem)) { + true + } else { + // check canonicalized + // find all attribs ref in all base expressions + val baseAttribs = elem.references + // collect all the list of canonicalized attributes for these base attribs + val substitutables = baseAttribs + .map(x => { + val seqContainingAttrib = this.attribRefBasedEquivalenceList + .filter(buff => buff.exists(_.canonicalized == x.canonicalized)) + if(!(seqContainingAttrib.isEmpty || seqContainingAttrib.size == 1)) { + val errorMessage = s"Attribute $x found in" + + s" more than 1 buffers" + throwExceptionOrLogWarning(errorMessage) + } + if (seqContainingAttrib.nonEmpty) { + x -> seqContainingAttrib.head.head + } else { + x -> null + } + }).filter{ case(_, replacement) => replacement ne null}.toMap + val canonicalizedExp = elem.transformUp { + case att: Attribute => substitutables.getOrElse(att, att) + } + + if (super.contains(canonicalizedExp)) { + true + } else { + val newCanonicalized = canonicalizedExp.transformUp { + case expr => + this.expressionBasedEquivalenceList + .find(buff => buff.exists(_.canonicalized == expr.canonicalized)) + .map(_.head) + .getOrElse(expr) + } + super.contains(newCanonicalized) + } + } + } + + /** + * This gives all the constraints whose references are subset of canonicalized + * attributes of interests + * @param expressionsOfInterest A sequence of expression for which constraints are desired & + * constraints should be such that its references are subset of + * the canonicalized version of attributes in the passed sequence + * @return Sequence of constraint expressions of compound types. + */ + override def getConstraintsSubsetOfAttributes(expressionsOfInterest: Iterable[Expression]): + Seq[Expression] = { + val canonicalAttribsMapping = expressionsOfInterest.map(expr => + (this.attribRefBasedEquivalenceList ++ this.expressionBasedEquivalenceList). + find(buff => buff.exists(_.canonicalized == expr.canonicalized)).map(buff => + buff.head -> expr).getOrElse(expr -> expr)).toMap + val refsOfInterest = canonicalAttribsMapping.keySet.map(_.references).reduce(_ ++ _) + this.originals.collect { + case expr if expr.references.subsetOf(refsOfInterest) & + expr.references.nonEmpty && expr.deterministic => expr + }.map(expr => expr.transformUp { + case x => canonicalAttribsMapping.getOrElse(x, x) + }).toSeq + } + + /** + * Consider a new filter generated out of constraints of the form + * IsNotNull(case....a....b...c) where the case expressions are + * complex. Since new filters generated out of constraints are always canonicalized + * it is possible that they are not compact as they are written in terms of + * basic attributes. If an alias to this complex expression is present, then it + * makes sense to rewrite the newly generated filter as IsNotNull(alias.attribute) + * to avoid expensive calculation, especially that we have large case optimization. + * This function simply tries to compact the expression where possible by replacing + * the expression with an alias's attribute. + * + * @param expr Expression to compact ( decanonicalize) + * @return Expression which is compacted, if possible + */ + override def rewriteUsingAlias(expr: Expression): Expression = + expr.transformDown { + case x: Attribute => x + case x => getDecanonicalizedAttributeForExpression(x) + } + + /** + * Decanonicalizes the NullIntolerant Expression. + * The need for this arises, because when spark is attempting to generate + * new NotNull constraints from the existing constraint, it may not return + * any NotNull constraints, if the underlying subexpression is not of type + * NullIntolerant. + * Consider following two cases: + * Lets say the base canonicalized constraint is of the form a + b > 5. + * A GreaterThan expression is NullIntolerant, so spark delves deep + * and finds, it is composed of a & b attributes, & thus returns two + * new IsNotNull constraints, namely IsNotNull(a) and IsNotNull(b). + * But if the base canonicalized constraint is of the form + * case(a....., b...) > 5, in this situation because case expression + * does not implement NullIntolerant, spark does not go deep & hence + * returns 0 not null constraints. + * This function handles this situation, by replacing an underlying + * canonicalized complex expression with an alias's attribute so + * that NotNull constraint can be generated. + * Thus case (a.....b) > 5 will be temporarily converted into z > 5. + * Once an IsNotNull(z) is returned as a new constraint, we store + * IsNotNull(z) in the Constraint set, again as canonicalized constraint, + * that is IsNotNull(case...a...b), which will ensure that pruning logic + * works fine. + * + * @return Set of constraint expressions where underlying NullIntolerant + * Expressions have been decanonicalized. + */ + override def getConstraintsWithDecanonicalizedNullIntolerant: ExpressionSet = { + def decanonicalizeNotNullIntolerant(expr: Expression): Expression = { + var foundNotNullIntolerant = false + expr.transformUp { + case x: LeafExpression => x + case x: NullIntolerant => if (foundNotNullIntolerant) { + val y = getDecanonicalizedAttributeForExpression(x) + if (y ne x) { + foundNotNullIntolerant = false + } + y + } else { + x + } + case x => val y = getDecanonicalizedAttributeForExpression(x) + if (y ne x) { + foundNotNullIntolerant = false + } else { + foundNotNullIntolerant = true + } + y + } + } + + new ConstraintSet(this.originals.map(expr => expr match { + case _: NullIntolerant => decanonicalizeNotNullIntolerant(expr) + case _ => expr + })) + } + + private def getDecanonicalizedAttributeForExpression(expr: Expression): Expression = { + val canonicalizedExpr = convertToCanonicalizedIfRequired(expr) + val bufferIndex = this.expressionBasedEquivalenceList.indexWhere(_.head == canonicalizedExpr) + if (bufferIndex != -1) { + // buffer size will always be > 1 + this.expressionBasedEquivalenceList(bufferIndex).last + } else { + expr + } + } + + private def throwExceptionOrLogWarning(errorMessage: String): Unit = { + if (Utils.isTesting) { + throw new IllegalStateException(errorMessage) + } else { + logWarning(errorMessage) + } + } + + override def toDebugString: String = + s""" + ${super.toDebugString} + |attribEquiList = ${this.attribRefBasedEquivalenceList.mkString(",")} + |exprEquivList = ${this.expressionBasedEquivalenceList.mkString(",")} + """.stripMargin + + override def toString(): String = + s""" + ${super.toString()} + |attribEquiList = ${this.attribRefBasedEquivalenceList.mkString(",")} + |exprEquivList = ${this.expressionBasedEquivalenceList.mkString(",")} + """.stripMargin +} + +object ConstraintSet { + def removeCanonicalizedExpressionFromBuffer( buff: mutable.Buffer[Expression], expr: Expression): + Unit = { + var keepGoing = true + while (keepGoing) { + val indx = buff.indexWhere(_.canonicalized == expr.canonicalized) + if (indx == -1) { + keepGoing = false + } else { + buff.remove(indx) + } + } + } + + private def addFiltersToConstraintSet(filters: GenTraversableOnce[Expression], + constraintSet: ConstraintSet): Unit = { + filters.foreach(expr => { + val conditionedElement = constraintSet.convertToCanonicalizedIfRequired(expr) + constraintSet.add(conditionedElement) + }) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala index 5d9b5be31195..27e3b25ec151 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala @@ -57,16 +57,14 @@ object ExpressionSet { * This is consistent with how we define `semanticEquals` between two expressions. */ class ExpressionSet protected( - private val baseSet: mutable.Set[Expression] = new mutable.HashSet, - private val originals: mutable.Buffer[Expression] = new ArrayBuffer) + protected val baseSet: mutable.Set[Expression] = new mutable.HashSet, + protected val originals: mutable.Buffer[Expression] = new ArrayBuffer) extends Iterable[Expression] { - // Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation. - protected def add(e: Expression): Unit = { if (!e.deterministic) { originals += e - } else if (!baseSet.contains(e.canonicalized)) { + } else if (!this.contains(e)) { baseSet.add(e.canonicalized) originals += e } @@ -82,55 +80,62 @@ class ExpressionSet protected( def contains(elem: Expression): Boolean = baseSet.contains(elem.canonicalized) override def filter(p: Expression => Boolean): ExpressionSet = { - val newBaseSet = baseSet.filter(e => p(e.canonicalized)) - val newOriginals = originals.filter(e => p(e.canonicalized)) - new ExpressionSet(newBaseSet, newOriginals) + val newBaseSet = baseSet.filter(e => p(e)) + val newOriginals = originals.filter(e => p(e)) + this.constructNew(newBaseSet, newOriginals) } override def filterNot(p: Expression => Boolean): ExpressionSet = { - val newBaseSet = baseSet.filterNot(e => p(e.canonicalized)) - val newOriginals = originals.filterNot(e => p(e.canonicalized)) + val newBaseSet = baseSet.filterNot(e => p(e)) + val newOriginals = originals.filterNot(e => p(e)) + this.constructNew(newBaseSet, newOriginals) + } + + def constructNew(newBaseSet: mutable.Set[Expression] = new mutable.HashSet, + newOriginals: mutable.Buffer[Expression] = new ArrayBuffer): ExpressionSet = { new ExpressionSet(newBaseSet, newOriginals) } def +(elem: Expression): ExpressionSet = { val newSet = clone() - newSet.add(elem) + newSet.add(this.convertToCanonicalizedIfRequired(elem)) newSet } def ++(elems: GenTraversableOnce[Expression]): ExpressionSet = { val newSet = clone() - elems.foreach(newSet.add) + elems.foreach(ele => newSet.add(this.convertToCanonicalizedIfRequired(ele))) newSet } def -(elem: Expression): ExpressionSet = { val newSet = clone() - newSet.remove(elem) + newSet.remove(this.convertToCanonicalizedIfRequired(elem)) newSet } def --(elems: GenTraversableOnce[Expression]): ExpressionSet = { val newSet = clone() - elems.foreach(newSet.remove) + elems.foreach(ele => newSet.remove(this.convertToCanonicalizedIfRequired(ele))) newSet } def map(f: Expression => Expression): ExpressionSet = { - val newSet = new ExpressionSet() - this.iterator.foreach(elem => newSet.add(f(elem))) + val newSet = this.constructNew() + this.iterator.foreach(elem => newSet.add(this.convertToCanonicalizedIfRequired(f(elem)))) newSet } + def flatMap(f: Expression => Iterable[Expression]): ExpressionSet = { - val newSet = new ExpressionSet() - this.iterator.foreach(f(_).foreach(newSet.add)) + val newSet = this.constructNew() + this.iterator.foreach(f(_).foreach(e => newSet.add(this.convertToCanonicalizedIfRequired(e)))) newSet } def iterator: Iterator[Expression] = originals.iterator + def union(that: ExpressionSet): ExpressionSet = { val newSet = clone() that.iterator.foreach(newSet.add) @@ -154,6 +159,28 @@ class ExpressionSet protected( override def clone(): ExpressionSet = new ExpressionSet(baseSet.clone(), originals.clone()) + def convertToCanonicalizedIfRequired(ele: Expression): Expression = ele + + def getConstraintsSubsetOfAttributes(expressionsOfInterest: Iterable[Expression]): + Seq[Expression] = Seq.empty + + def updateConstraints(outputAttribs: Seq[Attribute], + inputAttribs: Seq[Attribute], projectList: Seq[NamedExpression], + oldAliasedConstraintsCreator: Option[Seq[NamedExpression] => ExpressionSet]): + ExpressionSet = oldAliasedConstraintsCreator.map(aliasCreator => + this.union(aliasCreator(projectList))).getOrElse(this) + + def attributesRewrite(mapping: AttributeMap[Attribute]): ExpressionSet = + ExpressionSet(this.map(_ transform { + case a: Attribute => mapping(a) + })) + + def withNewConstraints(filters: ExpressionSet): ExpressionSet = ExpressionSet(filters) + + def rewriteUsingAlias(expr: Expression): Expression = expr + + def getConstraintsWithDecanonicalizedNullIntolerant: ExpressionSet = this + /** * Returns a string containing both the post [[Canonicalize]] expressions and the original * expressions in this set. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 53ac3560bc3b..83fbc54fffb0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -772,6 +772,8 @@ abstract class BinaryComparison extends BinaryOperator with Predicate { } protected lazy val ordering: Ordering[Any] = TypeUtils.getInterpretedOrdering(left.dataType) + + def reverseOperands(): BinaryComparison } @@ -828,6 +830,7 @@ case class EqualTo(left: Expression, right: Expression) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { defineCodeGen(ctx, ev, (c1, c2) => ctx.genEqual(left.dataType, c1, c2)) } + def reverseOperands(): BinaryComparison = this } // TODO: although map type is not orderable, technically map type should be able to be used @@ -888,6 +891,7 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp boolean ${ev.value} = (${eval1.isNull} && ${eval2.isNull}) || (!${eval1.isNull} && !${eval2.isNull} && $equalCode);""", isNull = FalseLiteral) } + def reverseOperands(): BinaryComparison = this } @ExpressionDescription( @@ -919,6 +923,7 @@ case class LessThan(left: Expression, right: Expression) override def symbol: String = "<" protected override def nullSafeEval(input1: Any, input2: Any): Any = ordering.lt(input1, input2) + def reverseOperands(): BinaryComparison = GreaterThan(right, left) } @ExpressionDescription( @@ -950,6 +955,7 @@ case class LessThanOrEqual(left: Expression, right: Expression) override def symbol: String = "<=" protected override def nullSafeEval(input1: Any, input2: Any): Any = ordering.lteq(input1, input2) + def reverseOperands(): BinaryComparison = GreaterThanOrEqual(right, left) } @ExpressionDescription( @@ -981,6 +987,8 @@ case class GreaterThan(left: Expression, right: Expression) override def symbol: String = ">" protected override def nullSafeEval(input1: Any, input2: Any): Any = ordering.gt(input1, input2) + + def reverseOperands(): BinaryComparison = LessThan(right, left) } @ExpressionDescription( @@ -1012,6 +1020,7 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) override def symbol: String = ">=" protected override def nullSafeEval(input1: Any, input2: Any): Any = ordering.gteq(input1, input2) + def reverseOperands(): BinaryComparison = LessThanOrEqual(right, left) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index aa8540fb4455..d37eacdae2aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -887,7 +887,7 @@ object InferFiltersFromGenerate extends Rule[LogicalPlan] { Seq( GreaterThan(Size(g.children.head), Literal(0)), IsNotNull(g.children.head) - ) + ).map(generate.child.constraints.convertToCanonicalizedIfRequired) ) -- generate.child.constraints if (inferredFilters.nonEmpty) { @@ -927,35 +927,49 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan] private def inferFilters(plan: LogicalPlan): LogicalPlan = plan transform { case filter @ Filter(condition, child) => val newFilters = filter.constraints -- - (child.constraints ++ splitConjunctivePredicates(condition)) - if (newFilters.nonEmpty) { - Filter(And(newFilters.reduce(And), condition), child) + (child.constraints ++ splitConjunctivePredicates(condition). + map(filter.constraints.convertToCanonicalizedIfRequired).toSet) + // do not directly use map on newFilters because the map function in ExpressionSet + // will reccanionicalize the expression + val decanonicalzedNewFilters = newFilters.iterator.map( + filter.constraints.rewriteUsingAlias(_)) + if (decanonicalzedNewFilters.nonEmpty) { + Filter(And(decanonicalzedNewFilters.reduce(And), condition), child) } else { filter } - case join @ Join(left, right, joinType, conditionOpt, _) => joinType match { // For inner join, we can infer additional filters for both sides. LeftSemi is kind of an - // inner join, it just drops the right side in the final output. + // inner join, it just dr ops the right side in the final output. case _: InnerLike | LeftSemi => - val allConstraints = getAllConstraints(left, right, conditionOpt) - val newLeft = inferNewFilter(left, allConstraints) - val newRight = inferNewFilter(right, allConstraints) + val (newLeft, newRight) = if (SQLConf.get.useOptimizedConstraintPropagation) { + inferAdditionalNewFilter(left, right.constraints, conditionOpt) -> + inferAdditionalNewFilter(right, left.constraints, conditionOpt) + } else { + val allConstraints = getAllConstraints(left, right, conditionOpt) + inferNewFilter(left, allConstraints) -> inferNewFilter(right, allConstraints) + } join.copy(left = newLeft, right = newRight) // For right outer join, we can only infer additional filters for left side. case RightOuter => - val allConstraints = getAllConstraints(left, right, conditionOpt) - val newLeft = inferNewFilter(left, allConstraints) + val newLeft = if (SQLConf.get.useOptimizedConstraintPropagation) { + inferAdditionalNewFilter(left, right.constraints, conditionOpt) + } else { + val allConstraints = getAllConstraints(left, right, conditionOpt) + inferNewFilter(left, allConstraints) + } join.copy(left = newLeft) - // For left join, we can only infer additional filters for right side. case LeftOuter | LeftAnti => - val allConstraints = getAllConstraints(left, right, conditionOpt) - val newRight = inferNewFilter(right, allConstraints) + val newRight = if (SQLConf.get.useOptimizedConstraintPropagation) { + inferAdditionalNewFilter(right, left.constraints, conditionOpt) + } else { + val allConstraints = getAllConstraints(left, right, conditionOpt) + inferNewFilter(right, allConstraints) + } join.copy(right = newRight) - case _ => join } } @@ -964,8 +978,8 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan] left: LogicalPlan, right: LogicalPlan, conditionOpt: Option[Expression]): ExpressionSet = { - val baseConstraints = left.constraints.union(right.constraints) - .union(ExpressionSet(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil))) + val baseConstraints = left.constraints.union(right.constraints). + union(ExpressionSet(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil))) baseConstraints.union(inferAdditionalConstraints(baseConstraints)) } @@ -981,6 +995,159 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan] Filter(newPredicates.reduce(And), plan) } } + + /** + * This function is similar to [[inferNewFilter()]] but it is used to create + * new filter of compound predicates for push down on the joining table. + * The issue is that when stock spark generates all the combination of + * constraints, (i.e with aliases as well as replacing the attribute of say + * LHS table with the joining RHS table's join condition), it replaces only one + * attribute in a constraint. As a result the constraints of the form of compound + * types ( i.e containing more than 1 attributes)are not created for push down. + * To elaborate: + * if LHS table is ( a, b, c) with projection (a1, a2, b, c) and RHS (x, y, z) + * with join condition a1 = x and b = y + * and say a constraint of the form a + b > 10 is present on LHS + * now stock spark will generate following constraints in expanded form + * a + b > 10, a1 +b > 10, a2 + b > 10 + * since a1 = x , it will also generate + * x + b > 10 + * and since b = y , it will generate + * a + y > 10, a1 + y > 10, a2 + y > 10 + * so the problem is that since it is replacing one join variable at a time, + * a compound constraint never gets created. (i.e x + y > 10). + * The below code explicitly asks the [[ConstraintSet.getConstraintsSubsetOfAttributes]] + * to return the substituted compound constraints if available, for push down. + * For stock spark the function is sort of no-op + * This function now completely replaces the stock spark's inferNewFilter + * for Join cases. + * The source of new filters can be grouped in 3 categories + * 1) New filters inferred from the conditions present in the join clause + * 2) New Filters inferred from the constraints of the other side of + * the join node + * 3) Not Null filters formed from #1 and #2 + * case 1 : consider a join condition of type + * a == x and b == y and c == z and d == a and d > 13 + * In the above case d and a are attributes of the same table. + * since d == a and a == x and since d > 13, it means x > 13 + * to get this new filer of type x > 13. the way it it is achieved is + 1) First using the base equalTo constraints, generate extra equalTo + * constraints which means using a == x, b == y, c == z and d == a, + * the [[inferAdditionalConstraints]], will yield a new equality + * constraint d == x + * 2) Using total equality constraints a == x, b == y, c == z + * and d == a and d ==x [[inferAdditionalConstraints]] , will then be able + * to generate x > 13 + * 3) Then using total equality constraints, use the otherSide's + * constraint to get new filters of single & compound type + * (including isnotnull) + * 4) Remove the trivial constraints which may come out of above, + * that is of the form P == P, but these need to be used to generate + * isNotNull constraints. + * 5) Using all the constraints available from above #4, + * generate any IsNotNull constraints. + * 6) For NotNull constraints, if the expression is not of + * type attribute, decanonicalize if possible. + * 7) All these constraints are checked and pruned if they are + * already contained in the thisSide constraint. + * + * @param plan Logical Plan on which pushdown filters are needed + * @param otherSideConstraint the constraints present on the other side of the join + * @param conditionOpt the conditions in the join clause + * @return + */ + private def inferAdditionalNewFilter(plan: LogicalPlan, otherSideConstraint: ExpressionSet, + conditionOpt: Option[Expression]): LogicalPlan = { + conditionOpt.map(condition => { + val predicates = splitConjunctivePredicates(condition) + val baseEqualityConstraints = predicates.filter(_.isInstanceOf[EqualTo]) + val extraEqualityConstraints = inferAdditionalConstraints( + ExpressionSet(baseEqualityConstraints)) + val totalEqualityConstraints = baseEqualityConstraints ++ extraEqualityConstraints + val extraOtherConstraints = inferAdditionalConstraints( + ExpressionSet(totalEqualityConstraints ++ predicates)) + + val allEqualityMappings = totalEqualityConstraints.map { case EqualTo(l, r) => (l, r) } + val planOutput = plan.outputSet + val validConstraintsFilter = (constraints: Iterable[Expression]) => constraints.filter(x => + x.references.subsetOf(planOutput) && !plan.constraints.contains(x)) + val isAttributeContainedInAttributeSet = (expr: Expression, superSet: AttributeSet) => + expr.references.subsetOf(superSet) + + val exprsOfOtherSide = allEqualityMappings.flatMap { case (l, r) => + if (!isAttributeContainedInAttributeSet(l, planOutput)) { + if (!isAttributeContainedInAttributeSet(r, planOutput)) { + Seq(l, r) + } else { + Seq(l) + } + } else if (!isAttributeContainedInAttributeSet(r, planOutput)) { + Seq(r) + } else { + Seq.empty[Expression] + } + }.toSet + val mappingsForThisSide = allEqualityMappings.filter{case (lhsExpr, rhsExpr) => + isAttributeContainedInAttributeSet(lhsExpr, planOutput) || + isAttributeContainedInAttributeSet(rhsExpr, planOutput)} + val totalNewFilters = if (exprsOfOtherSide.nonEmpty) { + val inferredFromOtherConstraint = inferNewConstraintsForThisSideFromOtherSideConstraints( + otherSideConstraint, exprsOfOtherSide, mappingsForThisSide) + // To generate non trivial constraints which are not of type isNotNull, + // the trivial constraints need to be removed. But for generating notnull + // constraints , even trivial constraints are needed. for eg + // if we have x = x as a condition, then the constraint is trivial, + // but it will still generate IsNotNull(x) constraint. + // this will be used to generate not null constraints + val (trivialConstraints, constraintsPart1) = inferredFromOtherConstraint.partition(expr => + expr match { + case EqualNullSafe(x, y) if x.canonicalized == y.canonicalized => true + case EqualTo(x, y) if x.canonicalized == y.canonicalized => true + case _ => false + }) + val constraintsOfInterest = validConstraintsFilter(ExpressionSet(extraOtherConstraints ++ + predicates ++ constraintsPart1 ++ extraEqualityConstraints)).flatMap(x => x match { + case IsNotNull(_: Attribute) => Seq(x) + case IsNotNull(exp: Expression) => validConstraintsFilter(inferIsNotNullConstraints( + plan.constraints.rewriteUsingAlias(exp))) + case _ => Seq(x) + }) + val newNotNulls = validConstraintsFilter(constructIsNotNullConstraints(ExpressionSet( + constraintsOfInterest ++ predicates ++ extraEqualityConstraints ++ + extraOtherConstraints ++ trivialConstraints), plan.output)) + constraintsOfInterest ++ newNotNulls + } else { + val constraintsOfInterest = validConstraintsFilter(ExpressionSet(extraOtherConstraints ++ + predicates ++ extraEqualityConstraints)) + constraintsOfInterest ++ validConstraintsFilter(constructIsNotNullConstraints( + ExpressionSet(constraintsOfInterest ++ predicates ++ extraEqualityConstraints ++ + extraOtherConstraints), plan.output)) + } + + if (totalNewFilters.isEmpty) { + plan + } else { + Filter(totalNewFilters.reduce(And), plan) + } + + }).getOrElse(plan) + } + + private def inferNewConstraintsForThisSideFromOtherSideConstraints( + otherSideConstraint: ExpressionSet, exprsOfOtherSide: Set[Expression], + mappingsForThisSide: Seq[(Expression, Expression)]): Seq[Expression] = { + otherSideConstraint.getConstraintsSubsetOfAttributes(exprsOfOtherSide).map( + expr => expr.transformUp { + case _expr: Expression => mappingsForThisSide.find { case (lhsExpr, rhsExpr) => + lhsExpr.canonicalized == _expr.canonicalized || + rhsExpr.canonicalized == _expr.canonicalized + }.map { case (lhsExpr, rhsExpr) => if (lhsExpr.canonicalized == _expr.canonicalized) { + rhsExpr + } else { + lhsExpr + }}.getOrElse(_expr) + }) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 57c3f3dbd050..bb8d8596f34a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -141,7 +141,8 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { } private def buildNewJoinType(filter: Filter, join: Join): JoinType = { - val conditions = splitConjunctivePredicates(filter.condition) ++ filter.constraints + val conditions = splitConjunctivePredicates(filter.condition). + map(filter.constraints.convertToCanonicalizedIfRequired) ++ filter.constraints val leftConditions = conditions.filter(_.references.subsetOf(join.left.outputSet)) val rightConditions = conditions.filter(_.references.subsetOf(join.right.outputSet)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 1a9c9d14e3ee..a4a8ff88511c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -191,7 +191,13 @@ abstract class UnaryNode extends LogicalPlan { allConstraints } - override protected lazy val validConstraints: ExpressionSet = child.constraints + override lazy val validConstraints: ExpressionSet = + if (!this.inputSet.subsetOf(this.outputSet)) { + child.constraints.updateConstraints(this.output, + child.output, Seq.empty[NamedExpression], None) + } else { + child.constraints + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala index c4243da7b9e4..ad62c994c243 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf trait QueryPlanConstraints extends ConstraintHelper { self: LogicalPlan => @@ -29,12 +30,14 @@ trait QueryPlanConstraints extends ConstraintHelper { self: LogicalPlan => */ lazy val constraints: ExpressionSet = { if (conf.constraintPropagationEnabled) { - validConstraints - .union(inferAdditionalConstraints(validConstraints)) - .union(constructIsNotNullConstraints(validConstraints, output)) - .filter { c => - c.references.nonEmpty && c.references.subsetOf(outputSet) && c.deterministic - } + val newConstraints = validConstraints.union( + inferAdditionalConstraints(validConstraints)).union(constructIsNotNullConstraints( + validConstraints.getConstraintsWithDecanonicalizedNullIntolerant, output)) + // Removed the criteria c.references.nonEmpty as it was causing a constraint of the + // of the form literal true or false being eliminated, causing idempotency check failure + newConstraints.filter(c => c.references.subsetOf(outputSet) && c.deterministic + && (c.references.nonEmpty || conf.useOptimizedConstraintPropagation)) + } else { ExpressionSet() } @@ -48,7 +51,9 @@ trait QueryPlanConstraints extends ConstraintHelper { self: LogicalPlan => * * See [[Canonicalize]] for more details. */ - protected lazy val validConstraints: ExpressionSet = ExpressionSet() + protected lazy val validConstraints: ExpressionSet = + if (SQLConf.get.useOptimizedConstraintPropagation) new ConstraintSet() + else ExpressionSet(Set.empty[Expression]) } trait ConstraintHelper { @@ -67,15 +72,17 @@ trait ConstraintHelper { val candidateConstraints = predicates - eq inferredConstraints ++= replaceConstraints(candidateConstraints, l, r) inferredConstraints ++= replaceConstraints(candidateConstraints, r, l) + case eq @ EqualTo(l @ Cast(_: Attribute, _, _), r: Attribute) => inferredConstraints ++= replaceConstraints(predicates - eq, r, l) + case eq @ EqualTo(l: Attribute, r @ Cast(_: Attribute, _, _)) => inferredConstraints ++= replaceConstraints(predicates - eq, l, r) + case _ => // No inference } inferredConstraints -- constraints } - private def replaceConstraints( constraints: ExpressionSet, source: Expression, @@ -106,7 +113,7 @@ trait ConstraintHelper { * Infer the Attribute-specific IsNotNull constraints from the null intolerant child expressions * of constraints. */ - private def inferIsNotNullConstraints(constraint: Expression): Seq[Expression] = + def inferIsNotNullConstraints(constraint: Expression): Seq[Expression] = constraint match { // When the root is IsNotNull, we can push IsNotNull through the child null intolerant // expressions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 91fb77574a0c..addd6c66d36d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -72,8 +72,8 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) !expressions.exists(!_.resolved) && childrenResolved && !hasSpecialExpressions } - override lazy val validConstraints: ExpressionSet = - getAllValidConstraints(projectList) + override lazy val validConstraints: ExpressionSet = child.constraints.updateConstraints( + this.output, child.output, this.projectList, Option(getAllValidConstraints)) } /** @@ -140,9 +140,9 @@ case class Filter(condition: Expression, child: LogicalPlan) override def maxRows: Option[Long] = child.maxRows - override protected lazy val validConstraints: ExpressionSet = { + override lazy val validConstraints: ExpressionSet = { val predicates = splitConjunctivePredicates(condition) - .filterNot(SubqueryExpression.hasCorrelatedSubquery) + .filterNot(SubqueryExpression.hasCorrelatedSubquery) child.constraints.union(ExpressionSet(predicates)) } } @@ -156,9 +156,7 @@ abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends Binar protected def rightConstraints: ExpressionSet = { require(left.output.size == right.output.size) val attributeRewrites = AttributeMap(right.output.zip(left.output)) - right.constraints.map(_ transform { - case a: Attribute => attributeRewrites(a) - }) + right.constraints.attributesRewrite(attributeRewrites) } override lazy val resolved: Boolean = @@ -185,8 +183,7 @@ case class Intersect( leftAttr.withNullability(leftAttr.nullable && rightAttr.nullable) } - override protected lazy val validConstraints: ExpressionSet = - leftConstraints.union(rightConstraints) + override lazy val validConstraints: ExpressionSet = leftConstraints.union(rightConstraints) override def maxRows: Option[Long] = { if (children.exists(_.maxRows.isEmpty)) { @@ -204,8 +201,7 @@ case class Except( override def nodeName: String = getClass.getSimpleName + ( if ( isAll ) "All" else "" ) /** We don't use right.output because those rows get excluded from the set. */ override def output: Seq[Attribute] = left.output - - override protected lazy val validConstraints: ExpressionSet = leftConstraints + override lazy val validConstraints: ExpressionSet = leftConstraints } /** Factory for constructing new `Union` nodes. */ @@ -287,14 +283,12 @@ case class Union( * mapping between the original and reference sequences are symmetric. */ private def rewriteConstraints( - reference: Seq[Attribute], - original: Seq[Attribute], - constraints: ExpressionSet): ExpressionSet = { + reference: Seq[Attribute], + original: Seq[Attribute], + constraints: ExpressionSet): ExpressionSet = { require(reference.size == original.size) val attributeRewrites = AttributeMap(original.zip(reference)) - constraints.map(_ transform { - case a: Attribute => attributeRewrites(a) - }) + constraints.attributesRewrite(attributeRewrites) } private def merge(a: ExpressionSet, b: ExpressionSet): ExpressionSet = { @@ -311,10 +305,10 @@ case class Union( common ++ others } - override protected lazy val validConstraints: ExpressionSet = { - children - .map(child => rewriteConstraints(children.head.output, child.output, child.constraints)) - .reduce(merge(_, _)) + override lazy val validConstraints: ExpressionSet = { + val unionConstraints = children.map(child => rewriteConstraints(children.head.output, + child.output, child.constraints)).reduce(merge(_, _)) + children.head.constraints.withNewConstraints(unionConstraints) } } @@ -343,7 +337,7 @@ case class Join( } } - override protected lazy val validConstraints: ExpressionSet = { + override lazy val validConstraints: ExpressionSet = { joinType match { case _: InnerLike if condition.isDefined => left.constraints @@ -362,8 +356,10 @@ case class Join( left.constraints case RightOuter => right.constraints - case _ => - ExpressionSet() + + case _ => if (SQLConf.get.useOptimizedConstraintPropagation) { + new ConstraintSet() + } else ExpressionSet() } } @@ -626,7 +622,8 @@ case class Aggregate( override lazy val validConstraints: ExpressionSet = { val nonAgg = aggregateExpressions.filter(_.find(_.isInstanceOf[AggregateExpression]).isEmpty) - getAllValidConstraints(nonAgg) + child.constraints.updateConstraints(this.output, + child.output, nonAgg, Option(getAllValidConstraints)) } } @@ -753,7 +750,9 @@ case class Expand( // This operator can reuse attributes (for example making them null when doing a roll up) so // the constraints of the child may no longer be valid. - override protected lazy val validConstraints: ExpressionSet = ExpressionSet() + override lazy val validConstraints: ExpressionSet = + if (SQLConf.get.useOptimizedConstraintPropagation) new ConstraintSet() + else ExpressionSet(Set.empty) } /** 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 078928391f56..8df19c9a1f25 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 @@ -216,6 +216,13 @@ object SQLConf { "for using switch statements in InSet must be non-negative and less than or equal to 600") .createWithDefault(400) + val OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED = + buildConf("spark.sql.optimizer.optimizedConstraintPropagation.enabled") + .internal() + .doc("use optimized algorithm for constraint propagation") + .booleanConf + .createWithDefault(true) + val PLAN_CHANGE_LOG_LEVEL = buildConf("spark.sql.planChangeLog.level") .internal() .doc("Configures the log level for logging the change from the original plan to the new " + @@ -3111,6 +3118,9 @@ class SQLConf extends Serializable with Logging { def optimizerInSetSwitchThreshold: Int = getConf(OPTIMIZER_INSET_SWITCH_THRESHOLD) + def useOptimizedConstraintPropagation: Boolean = + getConf(OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED) + def planChangeLogLevel: String = getConf(PLAN_CHANGE_LOG_LEVEL) def planChangeRules: Option[String] = getConf(PLAN_CHANGE_LOG_RULES) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index 79bd573f1d84..7148b8b7d63b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -197,8 +197,16 @@ class InferFiltersFromConstraintsSuite extends PlanTest { test("constraints should be inferred from aliased literals") { val originalLeft = testRelation.subquery('left).as("left") - val optimizedLeft = testRelation.subquery('left).where(IsNotNull('a) && 'a <=> 2).as("left") - + val optimizedLeft = if (SQLConf.get.useOptimizedConstraintPropagation) { + // The original commented code is slighly inefficient because if IsNotNull(a) is present + // then it should not form 'a <=> 2, but 'a === 2 ( i.e use EqualTo rather than + // EqualNullsafe + // val optimizedLeft = testRelation.subquery('left).where(IsNotNull('a) && 'a <=> 2). + // as("left") + testRelation.subquery('left).where(IsNotNull('a) && 'a === 2).as("left") + } else { + testRelation.subquery('left).where(IsNotNull('a) && 'a <=> 2).as("left") + } val right = Project(Seq(Literal(2).as("two")), testRelation.subquery('right)).as("right") val condition = Some("left.a".attr === "right.two".attr) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala index 5ad748b6113d..04f69e6a9de0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala @@ -19,6 +19,10 @@ package org.apache.spark.sql.catalyst.plans import java.util.TimeZone +import scala.collection.mutable + +import org.scalatest.Ignore + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -28,15 +32,23 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, DoubleType, IntegerType, LongType, StringType} + +/** + * This class is extended by OptimizedConstraintSuite and when it runs, all the tests + * of this class also be executed. The ignore is to prevent the tests from running twice + * unnecessarily + */ + +@Ignore class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { - private def resolveColumn(tr: LocalRelation, columnName: String): Expression = + def resolveColumn(tr: LocalRelation, columnName: String): Expression = resolveColumn(tr.analyze, columnName) - private def resolveColumn(plan: LogicalPlan, columnName: String): Expression = + def resolveColumn(plan: LogicalPlan, columnName: String): Expression = plan.resolveQuoted(columnName, caseInsensitiveResolution).get - private def verifyConstraints(found: ExpressionSet, expected: ExpressionSet): Unit = { + def verifyConstraints(found: ExpressionSet, expected: ExpressionSet): Unit = { val missing = expected -- found val extra = found -- expected if (missing.nonEmpty || extra.nonEmpty) { @@ -130,14 +142,21 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { assert(tr.where('c.attr > 10).select('a.as('x), 'b.as('y)).analyze.constraints.isEmpty) val aliasedRelation = tr.where('a.attr > 10).select('a.as('x), 'b, 'b.as('y), 'a.as('z)) - - verifyConstraints(aliasedRelation.analyze.constraints, - ExpressionSet(Seq(resolveColumn(aliasedRelation.analyze, "x") > 10, - IsNotNull(resolveColumn(aliasedRelation.analyze, "x")), - resolveColumn(aliasedRelation.analyze, "b") <=> resolveColumn(aliasedRelation.analyze, "y"), - resolveColumn(aliasedRelation.analyze, "z") <=> resolveColumn(aliasedRelation.analyze, "x"), - resolveColumn(aliasedRelation.analyze, "z") > 10, - IsNotNull(resolveColumn(aliasedRelation.analyze, "z"))))) + if(SQLConf.get.useOptimizedConstraintPropagation) { + verifyConstraints(ExpressionSet(aliasedRelation.analyze.constraints), + ExpressionSet(Seq(resolveColumn(aliasedRelation.analyze, "x") > 10, + IsNotNull(resolveColumn(aliasedRelation.analyze, "x"))))) + } else { + verifyConstraints(ExpressionSet(aliasedRelation.analyze.constraints), + ExpressionSet(Seq(resolveColumn(aliasedRelation.analyze, "x") > 10, + IsNotNull(resolveColumn(aliasedRelation.analyze, "x")), + resolveColumn(aliasedRelation.analyze, "b") <=> + resolveColumn(aliasedRelation.analyze, "y"), + resolveColumn(aliasedRelation.analyze, "z") <=> + resolveColumn(aliasedRelation.analyze, "x"), + resolveColumn(aliasedRelation.analyze, "z") > 10, + IsNotNull(resolveColumn(aliasedRelation.analyze, "z"))))) + } val multiAlias = tr.where('a === 'c + 10).select('a.as('x), 'c.as('y)) verifyConstraints(multiAlias.analyze.constraints, @@ -171,14 +190,21 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { .where('a.attr > 10) .union(tr2.where('d.attr > 11)) .analyze.constraints, - ExpressionSet(Seq(a > 10 || a > 11, IsNotNull(a)))) + new ConstraintSet(mutable.Buffer((a > 10 || a > 11), IsNotNull(a)))) val b = resolveColumn(tr1, "b") + + val cond1 = a > 10 || a > 11 + val cond2 = b < 10 || b < 11 + + cond1.canonicalized.deterministic + cond2.canonicalized.deterministic verifyConstraints(tr1 .where('a.attr > 10 && 'b.attr < 10) .union(tr2.where('d.attr > 11 && 'e.attr < 11)) .analyze.constraints, - ExpressionSet(Seq(a > 10 || a > 11, b < 10 || b < 11, IsNotNull(a), IsNotNull(b)))) + new ConstraintSet(mutable.Buffer(cond1, cond2, + IsNotNull(a), IsNotNull(b)))) } test("propagating constraints in intersect") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/OptimizedConstraintPropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/OptimizedConstraintPropagationSuite.scala new file mode 100644 index 000000000000..c541411b0213 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/OptimizedConstraintPropagationSuite.scala @@ -0,0 +1,1262 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans + +import org.junit.Assert._ + +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateSubqueryAliases, EmptyFunctionRegistry, FakeV2SessionCatalog} +import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{IsNotNull, _} +import org.apache.spark.sql.catalyst.optimizer.{CombineFilters, CombineUnions, InferFiltersFromConstraints, Optimizer, PruneFilters, PushDownPredicates, PushPredicateThroughJoin, PushProjectionThroughUnion} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, LongType} + +class OptimizedConstraintPropagationSuite extends ConstraintPropagationSuite { + + /** + * Default spark optimizer is not used in the tests as some of the tests were false passing. + * Many assertions go through fine hiding the bugs because of other rules in the optimizer. + * For eg., a test dedicated to test filter pruning ( involving aliases) & hence relying + * on contains function of ConstraintSet ( & indirectly the attributeEquivalenceList etc ) + * was false passing because of an optimizer rule, which replaces the alias with the actual + * expression in the plan. Combining Filter is commented just to be sure that ConstraintSet + * coming out of each node contains right the constraints & more importantly the + * attributeEquivalenceList & expressionEquivalenceList contains the right data. + * Otherwise it is possible that those Lists are empty & tests false passing + */ + + val trivialConstraintAbsenceChecker = (constraints: ExpressionSet) => assertTrue( + !constraints.exists(x => x match { + case EqualNullSafe(a, b) if a.canonicalized == b.canonicalized => true + case EqualTo(a, b) if a.canonicalized == b.canonicalized => true + case _ => false + })) + + test("checking number of base constraints in project node") { + assume(SQLConf.get.useOptimizedConstraintPropagation) + val tr = LocalRelation('a.int, 'b.string, 'c.int) + val y = tr.where('c.attr > 10).select('a.as('x), 'b.as('y), 'c, 'c.as('c1)).analyze + assert(y.resolved) + val constraints = y.constraints + trivialConstraintAbsenceChecker(constraints) + assertEquals(2, constraints.size) + + verifyConstraints(ExpressionSet(constraints), + ExpressionSet(Seq(resolveColumn(y.analyze, "c") > 10, + IsNotNull(resolveColumn(y.analyze, "c"))))) + } + + test("checking number of base constraints with " + + "filter dependent on multiple attributes") { + assume(SQLConf.get.useOptimizedConstraintPropagation) + val tr = LocalRelation('a.int, 'b.string, 'c.int) + val y = tr.where('c.attr + 'a.attr > 10).select('a, 'a.as('x), 'b.as('y), 'c, + 'c.as('c1)).analyze + assert(y.resolved) + val constraints = y.constraints + trivialConstraintAbsenceChecker(constraints) + assertEquals(3, constraints.size) + + verifyConstraints(ExpressionSet(constraints), + ExpressionSet(Seq( + resolveColumn(y.analyze, "c") + + resolveColumn(y.analyze, "a") > 10, + IsNotNull(resolveColumn(y.analyze, "c")), + IsNotNull(resolveColumn(y.analyze, "a"))))) + } + + test("checking filter pruning") { + assume(SQLConf.get.useOptimizedConstraintPropagation) + val tr = LocalRelation('a.int, 'b.string, 'c.int) + val y = tr.where('c.attr + 'a.attr > 10).select('a, 'a.as('x), 'b.as('y), 'c, + 'c.as('c1)).where('x.attr + 'c1.attr > 10).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING).execute(y) + val constraints = optimized.constraints + trivialConstraintAbsenceChecker(constraints) + assertEquals(3, constraints.size) + + verifyConstraints(ExpressionSet(constraints), + ExpressionSet(Seq( + resolveColumn(y.analyze, "c") + + resolveColumn(y.analyze, "a") > 10, + + IsNotNull(resolveColumn(y.analyze, "c")), + IsNotNull(resolveColumn(y.analyze, "a"))))) + val allFilters = optimized.collect[Filter] { + case x: Filter => x + } + assertEquals(1, allFilters.size) + val conditionalExps = allFilters.head.expressions.flatMap(expr => expr.collect { + case x: GreaterThan => x + case y: LessThan => y + }) + assertEquals(1, conditionalExps.size) + val correctAnswer = tr.where('c.attr + 'a.attr > 10 && IsNotNull('a) && IsNotNull('c)). + select('a, 'a.as('x), 'b.as('y), 'c, 'c.as('c1)).analyze + comparePlans(optimized, correctAnswer) + } + + test("filter pruning on Join Node") { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + val tr2 = LocalRelation('x.int, 'y.string, 'z.int) + val y = tr1.where('c.attr + 'a.attr > 10).select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, + 'c.as('c1)).join(tr2, Inner, Some("a2".attr === "x".attr)) + .where('a1.attr + 'c1.attr > 10).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING). + execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilters = optimized.collect[Filter] { + case x: Filter => x + } + + val conditionalExps = allFilters.flatMap(filter => + filter.expressions.flatMap(expr => expr.collect { + case x: GreaterThan => x + case y: LessThan => y + })) + assertEquals(1, conditionalExps.size) + val correctAnswer = tr1.where('c.attr + 'a.attr > 10 && IsNotNull('a) && IsNotNull('c)). + select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, + 'c.as('c1)).join(tr2.where(IsNotNull('x)), Inner, Some("a2".attr === "x".attr)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Alias in different projects have same exprID..!") { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int, 'd.int) + val y = tr1.where('c.attr + 'a.attr > 10).select('a, 'a.as('a1), + 'a.as('a2), 'b.as('b1), 'c, 'c.as('c1), Literal(1).as("one"), + Literal(1).as("one_") + ).where('b1.attr > 10).select( Literal(1).as("one"), + Literal(1).as("one_"), 'b1.attr). + where('one.attr > 1).analyze + var exprId1: Option[ExprId] = None + var exprId2: Option[ExprId] = None + val bugify = y.transformUp { + case p@Project(pl, child) => if (exprId1.isEmpty) { + exprId1 = pl.find(_.name == "one").map(_.asInstanceOf[Alias].exprId) + exprId2 = pl.find(_.name == "one_").map(_.asInstanceOf[Alias].exprId) + p + } else { + val newPl = pl.map(ne => if (ne.name == "one") { + val al = ne.asInstanceOf[Alias] + Alias(al.child, al.name)(exprId1.get) + } else if (ne.name == "one_") { + val al = ne.asInstanceOf[Alias] + Alias(al.child, al.name)(exprId2.get) + } else ne ) + Project(newPl, child) + } + case f: Filter if exprId1.isDefined => f.transformExpressionsUp { + case expr: Expression => expr.transformUp { + case at: AttributeReference if at.name == "one" => at.withExprId(exprId1.get) + case at: AttributeReference if at.name == "one_" => at.withExprId(exprId2.get) + } + } + }.analyze + + assert(bugify.resolved) + val optimized = GetOptimizer(OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING). + execute(bugify) + val topConstraint = optimized.constraints + // there should not be any trivial constraint present + trivialConstraintAbsenceChecker(topConstraint) + } + + test("new filter pushed down on Join Node") { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + val tr2 = LocalRelation('x.int, 'y.string, 'z.int) + val y = tr1.where('c.attr + 'a.attr > 10 && 'a.attr > -15).select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, + 'c.as('c1)).join(tr2, Inner, Some("a2".attr === "x".attr)) + .where('a1.attr + 'c1.attr > 10).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING). + execute(y) + val allFilters = optimized.collect[Filter] { + case x: Filter => x + } + val conditionalExps = allFilters.flatMap(filter => + filter.expressions.flatMap(expr => expr.collect { + case x: GreaterThan => x + case y: LessThan => y + })) + assertEquals(3, conditionalExps.size) + val correctAnswer = tr1.where('c.attr + 'a.attr > 10 && 'a.attr > -15 + && IsNotNull('a) && IsNotNull('c)).select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, + 'c.as('c1)).join(tr2.where(IsNotNull('x) && 'x.attr > -15), + Inner, Some("a2".attr === "x".attr)).analyze + trivialConstraintAbsenceChecker(optimized.constraints) + comparePlans(optimized, correctAnswer) + } + + test("new filter pushed down on Join Node with multiple join conditions") { + assume(SQLConf.get.useOptimizedConstraintPropagation) + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + val tr2 = LocalRelation('x.int, 'y.string, 'z.int) + tr1.where('c.attr + 'a.attr > 10 && 'a.attr > -15).select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, + 'c.as('c1)).join(tr2, Inner, Some("a2".attr === "x".attr && 'c1.attr === 'z.attr)) + .where('a1.attr + 'c1.attr > 10) + } + val (optimized, _) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING) + } + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilters = optimized.collect[Filter] { + case x: Filter => x + } + val conditionalExps = allFilters.flatMap(filter => + filter.expressions.flatMap(expr => expr.collect { + case x: GreaterThan => x + case y: LessThan => y + })) + assertEquals(4, conditionalExps.size) + + // there should be a + operator present on each side of the join node + val joinNode = optimized.collectFirst { + case j: Join => j + }.get + assertTrue(joinNode.left.collect { + case f: Filter => f + }.exists(f => f.condition.collectFirst { + case a: Add => a + }.isDefined)) + assertTrue(joinNode.right.collect { + case f: Filter => f + }.exists(f => f.condition.collectFirst { + case a: Add => a + }.isDefined)) + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + val tr2 = LocalRelation('x.int, 'y.string, 'z.int) + val correctAnswer = tr1.where('c.attr + 'a.attr > 10 && 'a.attr > -15 && + IsNotNull('a) && IsNotNull('c)).select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, + 'c.as('c1)).join(tr2.where(IsNotNull('x) && IsNotNull('z) && 'x.attr > -15 + && 'z.attr + 'x.attr > 10), + Inner, Some("a2".attr === "x".attr && 'c1.attr === 'z.attr)).analyze + + comparePlans(optimized, correctAnswer) + // get plan for stock spark + val (optimized1, _) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING) + } + // The plans don't match as stock spark does not push down a filter of form x + z > 10 + // comparePlans(optimized1, correctAnswer) + } + + test("filter pruning when original attributes are lost") { + val tr = LocalRelation('a.int, 'b.string, 'c.int) + val y = tr.where('c.attr + 'a.attr > 10).select('a, 'a.as('x), 'b.as('y), 'c, + 'c.as('c1)).select('x.as('x1), 'y.as('y1), + 'c1.as('c2)).where('x1.attr + 'c2.attr > 10).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING).execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilters = optimized.collect[Filter] { + case x: Filter => x + } + assertEquals(1, allFilters.size) + val conditionalExps = allFilters.head.expressions.flatMap(expr => expr.collect { + case x: GreaterThan => x + case y: LessThan => y + }) + assertEquals(1, conditionalExps.size) + val correctAnswer = tr.where('c.attr + 'a.attr > 10 && IsNotNull('a) && IsNotNull('c)). + select('a, 'a.as('x), 'b.as('y), 'c, + 'c.as('c1)).select('x.as('x1), 'y.as('y1), + 'c1.as('c2)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("filter pruning when partial attributes are lost") { + val tr = LocalRelation('a.int, 'b.string, 'c.int) + val y = tr.where('c.attr + 'a.attr > 10).select('a, 'a.as('x), 'b.as('y), 'c, + 'c.as('c1)).select('c, 'x.as('x1), 'y.as('y1), + 'c1.as('c2)).where('x1.attr + 'c.attr > 10).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING).execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilters = optimized.collect[Filter] { + case x: Filter => x + } + assertEquals(1, allFilters.size) + val conditionalExps = allFilters.head.expressions.flatMap(expr => expr.collect { + case x: GreaterThan => x + case y: LessThan => y + }) + assertEquals(1, conditionalExps.size) + val correctAnswer = tr.where('c.attr + 'a.attr > 10 && IsNotNull('a) && IsNotNull('c)). + select('a, 'a.as('x), 'b.as('y), 'c, + 'c.as('c1)).select('c, 'x.as('x1), 'y.as('y1), + 'c1.as('c2)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("filter pruning with expressions in alias") { + val tr = LocalRelation('a.int, 'b.string, 'c.int) + val y = tr.where('c.attr + 'a.attr > 10).select('a, ('a.attr + 'c.attr).as('x), + 'b.as('y), 'c, + 'c.as('c1)).select('c, 'x.as('x1), 'y.as('y1), + 'c1.as('c2)).where('x1.attr > 10).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING).execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilters = optimized.collect[Filter] { + case x: Filter => x + } + assertEquals(1, allFilters.size) + val conditionalExps = allFilters.head.expressions.flatMap(expr => expr.collect { + case x: GreaterThan => x + case y: LessThan => y + }) + assertEquals(1, conditionalExps.size) + val correctAnswer = tr.where('c.attr + 'a.attr > 10 && IsNotNull('a) && IsNotNull('c)). + select('a, ('a.attr + 'c.attr).as('x), + 'b.as('y), 'c, + 'c.as('c1)).select('c, 'x.as('x1), 'y.as('y1), + 'c1.as('c2)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("filter pruning with subexpressions in alias") { + val tr = LocalRelation('a.int, 'b.string, 'c.int) + val y = tr.where('c.attr + 'a.attr + 'b.attr > 10).select(('a.attr + 'c.attr).as('x), + 'b.as('y)).select('x.as('x1), 'y.as('y1)). + where('x1.attr + 'y1.attr > 10).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING).execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilters = optimized.collect[Filter] { + case x: Filter => x + } + assertEquals(1, allFilters.size) + val conditionalExps = allFilters.head.expressions.flatMap(expr => expr.collect { + case x: GreaterThan => x + case y: LessThan => y + }) + assertEquals(1, conditionalExps.size) + val correctAnswer = tr.where('c.attr + 'a.attr + 'b.attr > 10 && IsNotNull('a) && IsNotNull('c) + && IsNotNull('b)). + select(('a.attr + 'c.attr).as('x), + 'b.as('y)).select('x.as('x1), 'y.as('y1)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("filter pruning using expression equivalence list - #1") { + val tr = LocalRelation('a.int, 'b.string, 'c.int) + val y = tr.where('c.attr + 'a.attr + 'b.attr > 10).select('a, 'c, ('a.attr + 'c.attr).as('x), + 'b, 'b.as('y)).where('x.attr + 'b.attr > 10).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING).execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilters = optimized.collect[Filter] { + case x: Filter => x + } + val conditionalExps = allFilters.flatMap(_.expressions).flatMap(expr => expr.collect { + case x: GreaterThan => x + case y: LessThan => y + }) + assertEquals(1, conditionalExps.size) + val correctAnswer = tr.where('c.attr + 'a.attr + 'b.attr > 10 && IsNotNull('a) && IsNotNull('c) + && IsNotNull('b)). + select('a, 'c, ('a.attr + 'c.attr).as('x), + 'b, 'b.as('y)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("filter pruning using expression equivalence list - #2") { + val tr = LocalRelation('a.int, 'b.string, 'c.int) + val y = tr.where('c.attr + 'a.attr + 'b.attr > 10).select('c, ('a.attr + 'c.attr).as('x), + ('a.attr + 'c.attr).as('z), 'b, 'b.as('y)).where('x.attr + 'b.attr > 10). + select('z, 'y).where('z.attr + 'y.attr > 10).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING).execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilters = optimized.collect[Filter] { + case x: Filter => x + } + val conditionalExps = allFilters.flatMap(_.expressions).flatMap(expr => expr.collect { + case x: GreaterThan => x + case y: LessThan => y + }) + assertEquals(1, conditionalExps.size) + val correctAnswer = tr.where('c.attr + 'a.attr + 'b.attr > 10 && IsNotNull('a) && IsNotNull('c) + && IsNotNull('b)).select('c, ('a.attr + 'c.attr).as('x), + ('a.attr + 'c.attr).as('z), 'b, 'b.as('y)).select('z, 'y).analyze + + comparePlans(optimized, correctAnswer) + val z = tr.where('c.attr + 'a.attr + 'b.attr > 10).select('c, ('a.attr + 'c.attr).as('x), + ('a.attr + 'c.attr).as('z), 'b, 'b.as('y)).where('x.attr + 'b.attr > 10). + select('z, 'y).where('z.attr + 'y.attr > 10).select(('z.attr + 'y.attr).as('k)). + where('k.attr > 10).analyze + + val correctAnswer1 = tr.where('c.attr + 'a.attr + 'b.attr > 10 && IsNotNull('a) && IsNotNull('c) + && IsNotNull('b)).select('c, ('a.attr + 'c.attr).as('x), + ('a.attr + 'c.attr).as('z), 'b, 'b.as('y)).select('z, 'y). + select(('z.attr + 'y.attr).as('k)).analyze + + comparePlans(GetOptimizer(OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING).execute(z), correctAnswer1) + } + + test("check redundant constraints are not added") { + assume(SQLConf.get.useOptimizedConstraintPropagation) + val tr = LocalRelation('a.int, 'b.int, 'c.int, 'd.int) + val trAnalyzed = tr.analyze + val aliasedAnalyzed = trAnalyzed.where('c.attr + 'a.attr + 'b.attr > 10 && 'd.attr > 8). + select('a, 'd, 'd.attr.as('z), 'd.attr.as('z1), + ('a.attr + 'c.attr).as('x1), ('a.attr + 'c.attr).as('x), + 'b, 'b.as('y), 'c).analyze + val y = aliasedAnalyzed.where('x.attr + 'b.attr > 10 && 'z.attr > 8).analyze + assert(y.resolved) + /* total expected constraints + 1) a + c + b > 10 2) isnotnull(a) 3) isnotnull(b) 4) isnotnull(c) 5) d > 8 + 6) isnotnull(d) + */ + val expectedConstraints = ExpressionSet(Seq( + resolveColumn(trAnalyzed, "a") + resolveColumn(trAnalyzed, "b") + + resolveColumn(trAnalyzed, "c") > 10, + IsNotNull(resolveColumn(trAnalyzed, "a")), + IsNotNull(resolveColumn(trAnalyzed, "b")), + IsNotNull(resolveColumn(trAnalyzed, "c")), + IsNotNull(resolveColumn(trAnalyzed, "d")), + resolveColumn(trAnalyzed, "d") > 8)) + val constraints = y.constraints + trivialConstraintAbsenceChecker(constraints) + assertEquals(6, constraints.size) + verifyConstraints(constraints, expectedConstraints) + } + + test("new filter pushed down on Join Node with filter on each variable" + + " of join condition") { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + val tr1_ = tr1.where('c.attr + 'a.attr > 10 && 'a.attr > -11) + val tr2 = LocalRelation('x.int, 'y.string, 'z.int) + val tr2_ = tr2.where('x.attr > -12) + + val y = tr1_.select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, + 'c.as('c1)).join(tr2_.select('x.as('x1)), Inner, + Some('a2.attr === 'x1.attr)).where('a1.attr + 'c1.attr > 10).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING). + execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + val joinNode = optimized.find({ + case _: Join => true + case _ => false + }).get.asInstanceOf[Join] + + def checkForGreaterThanFunctions(node: LogicalPlan): Unit = { + val filterExps = node.collect { + case x: Filter => x + }.flatMap(_.expressions) + + assert(filterExps.exists(x => { + x.find { + case GreaterThan(_, Literal(-12, IntegerType)) => true + case _ => false + }.isDefined + })) + + assert(filterExps.exists(x => { + x.find { + case GreaterThan(_, Literal(-11, IntegerType)) => true + case _ => false + }.isDefined + })) + } + + checkForGreaterThanFunctions(joinNode.left) + checkForGreaterThanFunctions(joinNode.right) + + val allFilterExpressions = optimized.collect { + case x: Filter => x + }.flatMap(_.expressions) + assertEquals(5, allFilterExpressions.flatMap(_.collect { + case _: GreaterThan => true + }).size) + val correctAnswer = tr1.where('c.attr + 'a.attr > 10 && 'a.attr > -11 + && 'a.attr > -12 && IsNotNull('a) && IsNotNull('c)). + select('a, 'a.as('a1), 'a.as('a2), 'b.as('b1), + 'c, 'c.as('c1)).join(tr2.where('x.attr > -12 && IsNotNull('x) && 'x.attr > -11). + select('x.as('x1)), Inner, + Some('a2.attr === 'x1.attr)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("compound filter push down for left outer join") { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) + val tr2 = LocalRelation('x.int, 'y.int, 'z.int).subquery('tr2) + val y = tr1.where('a.attr + 'b.attr > 10) + .join(tr2.where('x.attr > 100), LeftOuter, Some("tr1.a".attr === "tr2.x".attr + && "tr1.b".attr === "tr2.y".attr)).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING). + execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + + val correctAnswer = tr1.where('a.attr + 'b.attr > 10 && IsNotNull('a) && IsNotNull('b)). + join(tr2.where('x.attr > 100 && IsNotNull('x) && IsNotNull('y) && 'x.attr + 'y.attr > 10), + LeftOuter, Some("tr1.a".attr === "tr2.x".attr && "tr1.b".attr === "tr2.y".attr)).analyze + comparePlans(optimized, correctAnswer) + } + + test("compound filter push down for left Anti join") { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) + val tr2 = LocalRelation('x.int, 'y.int, 'z.int).subquery('tr2) + val y = tr1.where('a.attr + 'b.attr > 10) + .join(tr2.where('x.attr > 100), LeftAnti, Some("tr1.a".attr === "tr2.x".attr + && "tr1.b".attr === "tr2.y".attr)).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING). + execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + + val correctAnswer = tr1.where('a.attr + 'b.attr > 10 && IsNotNull('a) && IsNotNull('b)). + join(tr2.where('x.attr > 100 && IsNotNull('x) && IsNotNull('y) && 'x.attr + 'y.attr > 10), + LeftAnti, Some("tr1.a".attr === "tr2.x".attr && "tr1.b".attr === "tr2.y".attr)).analyze + comparePlans(optimized, correctAnswer) + } + + test("compound filter push down for left Semi join") { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) + val tr2 = LocalRelation('x.int, 'y.int, 'z.int).subquery('tr2) + val y = tr1.where('a.attr + 'b.attr > 10) + .join(tr2.where('x.attr > 100), LeftSemi, Some("tr1.a".attr === "tr2.x".attr + && "tr1.b".attr === "tr2.y".attr)).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING). + execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + + val correctAnswer = tr1.where('a.attr + 'b.attr > 10 && IsNotNull('a) && IsNotNull('b) && + 'a.attr > 100). + join(tr2.where('x.attr > 100 && IsNotNull('x) && IsNotNull('y) && 'x.attr + 'y.attr > 10), + LeftSemi, Some("tr1.a".attr === "tr2.x".attr && "tr1.b".attr === "tr2.y".attr)).analyze + comparePlans(optimized, correctAnswer) + } + + test("compound filter push down for right outer join") { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) + val tr2 = LocalRelation('x.int, 'y.int, 'z.int).subquery('tr2) + val y = tr1.join(tr2.where('x.attr > 100 && 'x.attr + 'y.attr > 10), RightOuter, + Some("tr1.a".attr === "tr2.x".attr && "tr1.b".attr === "tr2.y".attr)).analyze + assert(y.resolved) + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING). + execute(y) + trivialConstraintAbsenceChecker(optimized.constraints) + val correctAnswer = tr1.where('a.attr + 'b.attr > 10 && IsNotNull('a) && IsNotNull('b) + && 'a.attr > 100).join(tr2.where('x.attr > 100 && IsNotNull('x) && IsNotNull('y) && + 'x.attr + 'y.attr > 10), RightOuter, Some("tr1.a".attr === "tr2.x".attr && + "tr1.b".attr === "tr2.y".attr)).analyze + comparePlans(optimized, correctAnswer) + } + + test("filter pruning due to new filter pushed down on Join Node ") { + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + val tr1_ = tr1.where('c.attr + 'a.attr > 10 && 'a.attr > -11) + val tr2 = LocalRelation('x.int, 'y.string, 'z.int) + val tr2_ = tr2.where('x.attr > -12) + tr1_.select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, + 'c.as('c1)).join(tr2_.select('x.as('x1)), Inner, + Some('a2.attr === 'x1.attr)).where('x1.attr + 'c1.attr > 10) + } + // The unanalyzed plan needs to be generated within the function + // so that sqlconf remains same within optimizer & outside + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING) + } + + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING) + } + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + assert(constraints2.size <= constraints1.size) + comparePlans(plan1, plan2) + } + + test("top filter should not be pruned for union with lower filter only on one table") { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int) + val tr2 = LocalRelation('d.int, 'e.int, 'f.int) + val tr3 = LocalRelation('g.int, 'h.int, 'i.int) + val y = tr1.where('a.attr > 10).union(tr2).union(tr3.where('g.attr > 10)) + val y1 = y.where('a.attr > 10).analyze + assert(y1.resolved) + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_UNIONS_PRUNING). + execute(y1) + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilterExpressions = optimized.collect { + case x: Filter => x + }.flatMap(_.expressions) + assert(allFilterExpressions.flatMap(_.collect { + case _: GreaterThan => true + }).size == 3) + val union = optimized.find { + case _: Union => true + case _ => false + }.get.asInstanceOf[Union] + + val numGTExpsBelowUnion = union.children.flatMap { + child => + child.expressions.flatMap(_.collect { + case x: GreaterThan => x + }) + } + assertEquals(3, numGTExpsBelowUnion.size) + + assert(union.children.forall(p => { + p.expressions.flatMap(_.collect { + case x: GreaterThan => x + }).nonEmpty + })) + val correctAnswer = new Union(Seq(tr1.where('a.attr > 10 && IsNotNull('a)), + tr2.where('d.attr > 10 && IsNotNull('d)), + tr3.where('g.attr > 10 && IsNotNull('g)))).analyze + + comparePlans(optimized, correctAnswer) + } + + test("top filter should be pruned for union with lower filter on all tables") { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int) + val tr2 = LocalRelation('d.int, 'e.int, 'f.int) + val tr3 = LocalRelation('g.int, 'h.int, 'i.int) + + val y = tr1.where('a.attr > 10).union(tr2.where('d.attr > 10)). + union(tr3.where('g.attr > 10)) + val y1 = y.where('a.attr > 10).analyze + assert(y1.resolved) + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_UNIONS_PRUNING). + execute(y1) + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilterExpressions = optimized.collect { + case x: Filter => x + }.flatMap(_.expressions) + assert(allFilterExpressions.flatMap(_.collect { + case _: GreaterThan => true + }).size == 3) + val union = optimized.find { + case _: Union => true + case _ => false + }.get.asInstanceOf[Union] + + assert(union.children.forall(p => { + p.expressions.flatMap(_.collect { + case x: GreaterThan => x + }).nonEmpty + })) + + val correctAnswer = new Union(Seq(tr1.where('a.attr > 10 && IsNotNull('a)), + tr2.where('d.attr > 10 && IsNotNull('d)), + tr3.where('g.attr > 10 && IsNotNull('g)))).analyze + + comparePlans(optimized, correctAnswer) + } + + test("top filter should be pruned for Intersection with lower filter on one or more tables") { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int) + val tr2 = LocalRelation('d.int, 'e.int, 'f.int) + val tr3 = LocalRelation('g.int, 'h.int, 'i.int) + + val y = tr1.where('a.attr > 10).intersect(tr2.where('e.attr > 5), isAll = true). + intersect(tr3.where('i.attr > -5), isAll = true) + + val y1 = y.select('a.attr.as("a1"), 'b.attr.as("b1"), 'c.attr.as("c1")).analyze + assert(y1.resolved) + + val y2 = y1.where('a1.attr > 10 && 'b1.attr > 5 && 'c1.attr > -5).analyze + assert(y2.resolved) + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_UNIONS_PRUNING). + execute(y2) + trivialConstraintAbsenceChecker(optimized.constraints) + val allFilterExpressions = optimized.collect { + case x: Filter => x + }.flatMap(_.expressions) + + assert(allFilterExpressions.flatMap(_.collect { + case _: GreaterThan => true + }).size == 3) + val correctAnswer = tr1.where(IsNotNull('a) && 'a.attr > 10). + intersect(tr2.where(IsNotNull('e) && 'e.attr > 5), isAll = true). + intersect(tr3.where(IsNotNull('i) && 'i.attr > -5), isAll = true). + select('a.attr.as("a1"), 'b.attr.as("b1"), + 'c.attr.as("c1")).analyze + + comparePlans(optimized, correctAnswer) + } + + test("top filter should be pruned for aggregate with lower filter") { + val tr = LocalRelation('a.int, 'b.string, 'c.int, 'd.int) + assert(tr.analyze.constraints.isEmpty) + val aliasedRelation = tr.where('c.attr > 10 && 'a.attr < 5) + .groupBy('a, 'c, 'b)('a, 'c.as("c1"), count('a).as("a3")). + select('c1, 'a, 'a3).analyze + val withTopFilter = aliasedRelation.where('a.attr < 5 && 'c1.attr > 10 && 'a3.attr > 20).analyze + val optimized = GetOptimizer(OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING). + execute(withTopFilter) + trivialConstraintAbsenceChecker(optimized.constraints) + val correctAnswer = tr.where('c.attr > 10 && 'a.attr < 5 && IsNotNull('a) && IsNotNull('c) + ).groupBy('a, 'c, 'b)('a, 'c.as("c1"), count('a).as("a3")). + where('a3 > Literal(20).cast(LongType)).select('c1, 'a, 'a3).analyze + comparePlans(correctAnswer, optimized) + } + + test("Duplicate removed attributes with different metadata causes assert failure") { + val tr = LocalRelation('a.int, 'b.string, 'c.int, 'd.int) + val aliasedRelation = tr.select('a, 'a.as("a1"), 'a, 'a.as("a2"), + 'a.as("a3"), 'c, 'c.as("c1")).select('c1, 'a3).where('a3 > 5). + analyze + val bugify = aliasedRelation.transformUp { + case Project(projList, child) if projList.exists(_.name == "a") => + Project(projList.zipWithIndex.map{ case(ne, i) => + ne match { + case att: AttributeReference => att.withQualifier(Seq(i.toString)) + case _ => ne + } + }, child) + } + GetOptimizer(OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING).execute(bugify) + } + + test("filter push down on join with aggregate") { + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + val tr2 = LocalRelation('x.int, 'y.string, 'z.int) + tr1.where('c.attr + 'a.attr > 10 && 'a.attr > -15).select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1)). + groupBy('b1.attr, 'c1.attr)('b1, 'c1.as("c2"), count('a).as("a3")). + select('c2, 'a3).join(tr2.where('x.attr > 9), Inner, Some("c2".attr === "x".attr)) + } + + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING) + } + + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING) + } + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + assert(constraints2.size <= constraints1.size) + comparePlans(plan1, plan2) + + val conditionFinder: PartialFunction[LogicalPlan, Seq[Expression]] = { + case f: Filter => f.expressions.find(x => x.find { + case GreaterThan(att: Attribute, Literal(9, IntegerType)) if att.name == "c" => true + case LessThan(Literal(9, IntegerType), att: Attribute) if att.name == "c" => true + case _ => false + }.isDefined).map(Seq(_)).getOrElse(Seq.empty[Expression]) + } + val result1 = plan1.collect { + conditionFinder + }.flatten + assert(result1.nonEmpty) + val result2 = plan2.collect { + conditionFinder + }.flatten + assert(result2.nonEmpty) + } + + // Not comparing with stock spark plan as stock spark plan is not optimal + test("test pruning using constraints with filters after project - 1") { + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + tr1.select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1)).where('c.attr + 'a.attr > 10 && 'a.attr > -15). + where('c1.attr + 'a2.attr > 10 && 'a2.attr > -15) + } + + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + val correctAnswer = LocalRelation('a.int, 'b.string, 'c.int). + select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1)).where('c.attr + 'a.attr > 10 && 'a.attr > -15 + && IsNotNull('a) && IsNotNull('c)).analyze + comparePlans(correctAnswer, plan2) + } + + // Not comparing with stock spark plan as stock spark plan is not optimal + test("test pruning using constraints with filters after project - 2") { + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + tr1.select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1)).where('c.attr + 'a.attr > 10 && 'a.attr > -15). + where('c1.attr + 'a2.attr > 10 && 'a2.attr > -15) + } + + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + val correctAnswer = LocalRelation('a.int, 'b.string, 'c.int). + select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1)).where('c.attr + 'a.attr > 10 && 'a.attr > -15 + && IsNotNull('a) && IsNotNull('c)).analyze + comparePlans(correctAnswer, plan2) + } + + // Not comparing with stock spark plan as stock spark plan is not optimal + test("test pruning using constraints with filters after project - 3") { + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + tr1.select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1)).where('c1.attr + 'a1.attr > 10 && 'a2.attr > -15). + where('c.attr + 'a.attr > 10 && 'a .attr > -15) + } + + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + val correctAnswer = LocalRelation('a.int, 'b.string, 'c.int). + select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1)).where('c1.attr + 'a1.attr > 10 && 'a2.attr > -15 + && IsNotNull('a) && IsNotNull('c)).analyze + comparePlans(correctAnswer, plan2) + } + + test("test new filter inference with decanonicalization for expression not" + + " implementing NullIntolerant - 1") { + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int) + tr1.select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1), + CaseWhen(Seq(('a.attr + 'b.attr + 'c.attr > Literal(1), + Literal(1)), ('a.attr + 'b.attr + 'c.attr > Literal(2), Literal(2))), + Option(Literal(null))).as("z")).where('z.attr > 10 && 'a2.attr > -15). + where(CaseWhen(Seq(('a.attr + 'b.attr + 'c.attr > Literal(1), + Literal(1)), ('a.attr + 'b.attr + 'c.attr > Literal(2), Literal(2))), + Option(Literal(null))) > 10 && 'a.attr > -15).where('z.attr > 10) + } + + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + val correctAnswer = LocalRelation('a.int, 'b.int, 'c.int). + select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1), CaseWhen(Seq( + ('a.attr + 'b.attr + 'c.attr > Literal(1), + Literal(1)), ('a.attr + 'b.attr + 'c.attr > Literal(2), Literal(2))), + Option(Literal(null))).as("z"), 'b).where('z.attr > 10 && 'a2.attr > -15 + && IsNotNull('a) && IsNotNull('z)).select('a, 'a1, 'a2, + 'b1, 'c, 'c1, 'z).analyze + comparePlans(correctAnswer, plan2) + } + + test("test new filter inference with decanonicalization for expression not" + + " implementing NullIntolerant - 2") { + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int) + tr1.select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1), + ('a.attr + CaseWhen(Seq(('a.attr + 'b.attr + 'c.attr > Literal(1), + Literal(1)), ('a.attr + 'b.attr + 'c.attr > Literal(2), Literal(2))), + Option(Literal(null)))).as("z")).where('z.attr > 10 && 'a2.attr > -15). + where('a.attr + CaseWhen(Seq(('a.attr + 'b.attr + 'c.attr > Literal(1), + Literal(1)), ('a.attr + 'b.attr + 'c.attr > Literal(2), Literal(2))), + Option(Literal(null))) > 10 && 'a.attr > -15).where('z.attr > 10) + } + + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + val correctAnswer = LocalRelation('a.int, 'b.int, 'c.int). + select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1), ('a.attr + CaseWhen(Seq( + ('a.attr + 'b.attr + 'c.attr > Literal(1), + Literal(1)), ('a.attr + 'b.attr + 'c.attr > Literal(2), Literal(2))), + Option(Literal(null)))).as("z"), 'b).where('z.attr > 10 && 'a2.attr > -15 + && IsNotNull('a) && IsNotNull('z)).select('a, 'a1, 'a2, + 'b1, 'c, 'c1, 'z).analyze + comparePlans(correctAnswer, plan2) + } + + test("test new filter inference with decanonicalization for expression" + + "implementing NullIntolerant") { + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int) + tr1.select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1), + ('a.attr + 'b.attr + 'c.attr ).as("z")).where('z.attr > 10 && 'a2.attr > -15). + where('a.attr + 'b1.attr + 'c.attr > 10 && 'a.attr > -15) + } + + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + val correctAnswer = LocalRelation('a.int, 'b.int, 'c.int). + select('a, 'a.as('a1), 'a.as('a2), + 'b.as('b1), 'c, 'c.as('c1), + ('a.attr + 'b.attr + 'c.attr ).as("z")).where('z.attr > 10 && 'a2.attr > -15 + && IsNotNull('a) && IsNotNull('b1) && IsNotNull('c)).analyze + comparePlans(correctAnswer, plan2) + } + + test("test pruning using constraints with filters after project with expression in" + + " alias.") { + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.int, 'c.int) + tr1.select('a, 'a.as('a1), 'a.as('a2), 'b, + 'b.as('b1), 'c, 'c.as('c1), ('a.attr + 'b.attr).as("z")). + where('c1.attr + 'z.attr > 10 && + 'a2.attr > -15). + where('c.attr + 'a.attr + 'b.attr > 10 && + 'a.attr > -15) + } + + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING) + } + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + val correctAnswer = LocalRelation('a.int, 'b.int, 'c.int). + select('a, 'a.as('a1), 'a.as('a2), 'b, + 'b.as('b1), 'c, 'c.as('c1), ('a.attr + 'b.attr).as("z")). + where('c1.attr + 'z.attr > 10 && + 'a2.attr > -15 && IsNotNull('b) + && IsNotNull('a) && IsNotNull('c)).analyze + comparePlans(correctAnswer, plan2) + } + + ignore("Disabled due to spark's canonicalization bug." + + " test pruning using constraints with filters after project with expression in alias.") { + + def getTestPlan: LogicalPlan = { + val tr1 = LocalRelation('a.int, 'b.string, 'c.int) + tr1.select('a, 'a.as('a1), 'a.as('a2), 'b, + 'b.as('b1), 'c, 'c.as('c1), ('a.attr + 'b.attr).as("z")). + where('c1.attr + 'z.attr > 10 && 'a2.attr > -15). + where('c.attr + 'a.attr + 'b.attr > 10 && 'a.attr > -15) + } + + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING) + } + + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING) + } + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + val correctAnswer = LocalRelation('a.int, 'b.string, 'c.int). + select('a, 'a.as('a1), 'a.as('a2), 'b, + 'b.as('b1), 'c, 'c.as('c1), ('a.attr + 'b.attr).as("z")). + where('c1.attr + 'z.attr > 10 && 'a2.attr > -15 + && IsNotNull('a) && IsNotNull('c) && IsNotNull('b)).analyze + comparePlans(correctAnswer, plan2) + } + + test("plan equivalence with case statements and performance comparison with benefit" + + "of more than 10x conservatively") { + def getTestPlan: LogicalPlan = { + val tr = LocalRelation('a.int, 'b.int, 'c.int, 'd.int, 'e.int, 'f.int, 'g.int, 'h.int, 'i.int, + 'j.int, 'k.int, 'l.int, 'm.int, 'n.int) + tr.select('a, 'b, 'c, 'd, 'e, 'f, 'g, 'h, 'i, 'j, 'k, 'l, 'm, 'n, + CaseWhen(Seq(('a.attr + 'b.attr + 'c.attr + 'd.attr + 'e.attr + 'f.attr + 'g.attr + + 'h.attr + 'i.attr + 'j.attr + 'k.attr + 'l.attr + 'm.attr + 'n.attr > Literal(1), + Literal(1)), + ('a.attr + 'b.attr + 'c.attr + 'd.attr + 'e.attr + 'f.attr + 'g.attr + 'h.attr + + 'i.attr + 'j.attr + 'k.attr + 'l.attr + 'm.attr + 'n.attr > Literal(2), Literal(2))), + Option(Literal(0))).as("JoinKey1") + ).select('a.attr.as("a1"), 'b.attr.as("b1"), 'c.attr.as("c1"), + 'd.attr.as("d1"), 'e.attr.as("e1"), 'f.attr.as("f1"), + 'g.attr.as("g1"), 'h.attr.as("h1"), 'i.attr.as("i1"), + 'j.attr.as("j1"), 'k.attr.as("k1"), 'l.attr.as("l1"), + 'm.attr.as("m1"), 'n.attr.as("n1"), 'JoinKey1.attr.as("cf1"), + 'JoinKey1.attr).select('a1, 'b1, 'c1, 'd1, 'e1, 'f1, 'g1, 'h1, 'i1, 'j1, 'k1, + 'l1, 'm1, 'n1, 'cf1, 'JoinKey1).join(tr, condition = Option('a.attr <=> 'JoinKey1.attr)) + } + val t1 = System.currentTimeMillis() + val (plan1, constraints1) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "false") { + executePlan(getTestPlan, OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING) + } + val t2 = System.currentTimeMillis() + val (plan2, constraints2) = withSQLConf[(LogicalPlan, ExpressionSet)]( + SQLConf.OPTIMIZER_CONSTRAINT_PROPAGATION_OPTIMIZED.key -> "true") { + executePlan(getTestPlan, OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING) + } + val t3 = System.currentTimeMillis() + trivialConstraintAbsenceChecker(constraints2) + assert(constraints1 match { + case _: ConstraintSet => false + case _: ExpressionSet => true + }) + + assert(constraints2 match { + case _: ConstraintSet => true + case _: ExpressionSet => false + }) + comparePlans(plan1, plan2) + val timeTakenPlan1 = t2 - t1 + val timeTakenPlan2 = t3 - t2 + // scalastyle:off println + println(s"Time taken to compile with constraint optimization off = $timeTakenPlan1 ms") + println(s"Time taken to compile with constraint optimization on = $timeTakenPlan2 ms") + // scalastyle:on println + assert(timeTakenPlan2 < timeTakenPlan1/10) + } + + def executePlan(plan: LogicalPlan, optimizerType: OptimizerTypes.Value): + (LogicalPlan, ExpressionSet) = { + object SimpleAnalyzer extends Analyzer( + new CatalogManager(FakeV2SessionCatalog, + new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, + SQLConf.get))) + + val optimizedPlan = GetOptimizer(optimizerType, Some(SQLConf.get)). + execute(SimpleAnalyzer.execute(plan)) + (optimizedPlan, optimizedPlan.constraints) + } +} + +object OptimizerTypes extends Enumeration { + val WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING, NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING, + NO_PUSH_DOWN_ONLY_PRUNING, WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_UNIONS_PRUNING = Value +} + +object GetOptimizer { + def apply(optimizerType: OptimizerTypes.Value, useConf: Option[SQLConf] = None): Optimizer = + optimizerType match { + case OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_PRUNING => + new Optimizer( new CatalogManager( + FakeV2SessionCatalog, + new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, + useConf.getOrElse(SQLConf.get)))) { + override def defaultBatches: Seq[Batch] = + Batch("Subqueries", Once, + EliminateSubqueryAliases) :: + Batch("Filter Pushdown and Pruning", FixedPoint(100), + PushPredicateThroughJoin, + PushDownPredicates, + InferFiltersFromConstraints, + CombineFilters, + PruneFilters) :: Nil + + override def nonExcludableRules: Seq[String] = Seq.empty[String] + } + case OptimizerTypes.NO_PUSH_DOWN_NO_COMBINE_FILTERS_ONLY_PRUNING => + new Optimizer( new CatalogManager( + FakeV2SessionCatalog, + new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, + useConf.getOrElse(SQLConf.get)))) { + override def defaultBatches: Seq[Batch] = + Batch("Subqueries", Once, + EliminateSubqueryAliases) :: + Batch("Filter Pruning", Once, + InferFiltersFromConstraints, + PruneFilters) :: Nil + + override def nonExcludableRules: Seq[String] = Seq.empty[String] + } + case OptimizerTypes.NO_PUSH_DOWN_ONLY_PRUNING => + new Optimizer( new CatalogManager( + FakeV2SessionCatalog, + new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, + useConf.getOrElse(SQLConf.get)))) { + override def defaultBatches: Seq[Batch] = + Batch("Subqueries", Once, + EliminateSubqueryAliases) :: + Batch("Filter Pruning", Once, + InferFiltersFromConstraints, + CombineFilters, + PruneFilters) :: Nil + + override def nonExcludableRules: Seq[String] = Seq.empty[String] + } + + case OptimizerTypes.WITH_FILTER_PUSHDOWN_THRU_JOIN_AND_UNIONS_PRUNING => + new Optimizer( new CatalogManager( + FakeV2SessionCatalog, + new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, + useConf.getOrElse(SQLConf.get)))) { + override def defaultBatches: Seq[Batch] = + Batch("Subqueries", Once, + EliminateSubqueryAliases) :: + Batch("Union Pushdown", FixedPoint(100), + CombineUnions, + PushProjectionThroughUnion, + PushDownPredicates, + InferFiltersFromConstraints, + CombineFilters, + PruneFilters) :: Nil + + override def nonExcludableRules: Seq[String] = Seq.empty[String] + } + } + } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 7c70ab98e418..6f81516f72b9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -63,7 +63,7 @@ trait PlanTestBase extends PredicateHelper with SQLHelper with SQLConfHelper { s * Since attribute references are given globally unique ids during analysis, * we must normalize them to check if two different queries are identical. */ - protected def normalizeExprIds(plan: LogicalPlan) = { + protected def normalizeExprIds(plan: LogicalPlan) = plan transformAllExpressions { case s: ScalarSubquery => s.copy(exprId = ExprId(0)) @@ -82,7 +82,6 @@ trait PlanTestBase extends PredicateHelper with SQLHelper with SQLConfHelper { s case udf: PythonUDF => udf.copy(resultId = ExprId(0)) } - } protected def rewriteNameFromAttrNullability(plan: LogicalPlan): LogicalPlan = { plan.transformAllExpressions { @@ -102,8 +101,8 @@ trait PlanTestBase extends PredicateHelper with SQLHelper with SQLConfHelper { s protected def normalizePlan(plan: LogicalPlan): LogicalPlan = { plan transform { case Filter(condition: Expression, child: LogicalPlan) => - Filter(splitConjunctivePredicates(condition).map(rewriteBinaryComparison) - .sortBy(_.hashCode()).reduce(And), child) + Filter(splitConjunctivePredicates(condition).map(rewriteEqualAndComparisons). + sortBy(_.hashCode()).reduce(And), child) case sample: Sample => sample.copy(seed = 0L) case Join(left, right, joinType, condition, hint) if condition.isDefined => @@ -115,8 +114,8 @@ trait PlanTestBase extends PredicateHelper with SQLHelper with SQLConfHelper { s } val newCondition = - splitConjunctivePredicates(condition.get).map(rewriteBinaryComparison) - .sortBy(_.hashCode()).reduce(And) + splitConjunctivePredicates(condition.get).map(rewriteEqualAndComparisons). + sortBy(_.hashCode()).reduce(And) Join(left, right, newJoinType, Some(newCondition), hint) } } @@ -128,13 +127,14 @@ trait PlanTestBase extends PredicateHelper with SQLHelper with SQLConfHelper { s * 2. (a <=> b), (b <=> a). * 3. (a > b), (b < a) */ - private def rewriteBinaryComparison(condition: Expression): Expression = condition match { + private def rewriteEqualAndComparisons(condition: Expression): Expression = condition match { case EqualTo(l, r) => Seq(l, r).sortBy(_.hashCode()).reduce(EqualTo) case EqualNullSafe(l, r) => Seq(l, r).sortBy(_.hashCode()).reduce(EqualNullSafe) case GreaterThan(l, r) if l.hashCode() > r.hashCode() => LessThan(r, l) case LessThan(l, r) if l.hashCode() > r.hashCode() => GreaterThan(r, l) case GreaterThanOrEqual(l, r) if l.hashCode() > r.hashCode() => LessThanOrEqual(r, l) case LessThanOrEqual(l, r) if l.hashCode() > r.hashCode() => GreaterThanOrEqual(r, l) + case br@BinaryComparison(lhs: Literal, rhs: Expression) => br.reverseOperands() case _ => condition // Don't reorder. } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala index 5deab79b9f67..af8dd4b8350a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala @@ -35,7 +35,7 @@ trait SQLHelper { * Sets all SQL configurations specified in `pairs`, calls `f`, and then restores all SQL * configurations. */ - protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + protected def withSQLConf[T](pairs: (String, String)*)(f: => T): T = { val conf = SQLConf.get val (keys, values) = pairs.unzip val currentValues = keys.map { key => diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/SPARK-33152_explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/SPARK-33152_explain.txt new file mode 100644 index 000000000000..4ee24b329997 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/SPARK-33152_explain.txt @@ -0,0 +1,752 @@ +== Physical Plan == +TakeOrderedAndProject (138) ++- * Project (137) + +- * SortMergeJoin Inner (136) + :- * Sort (74) + : +- Exchange (73) + : +- * HashAggregate (72) + : +- Exchange (71) + : +- * HashAggregate (70) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- Union (66) + : :- * HashAggregate (47) + : : +- Exchange (46) + : : +- * HashAggregate (45) + : : +- Union (44) + : : :- * Project (25) + : : : +- SortMergeJoin LeftOuter (24) + : : : :- * Sort (18) + : : : : +- Exchange (17) + : : : : +- * Project (16) + : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : :- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * Filter (3) + : : : : : : +- * ColumnarToRow (2) + : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * Project (7) + : : : : : +- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.item (4) + : : : : +- BroadcastExchange (14) + : : : : +- * Filter (13) + : : : : +- * ColumnarToRow (12) + : : : : +- Scan parquet default.date_dim (11) + : : : +- * Sort (23) + : : : +- Exchange (22) + : : : +- * Filter (21) + : : : +- * ColumnarToRow (20) + : : : +- Scan parquet default.catalog_returns (19) + : : +- * Project (43) + : : +- SortMergeJoin LeftOuter (42) + : : :- * Sort (36) + : : : +- Exchange (35) + : : : +- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * Project (31) + : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : :- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.store_sales (26) + : : : : +- ReusedExchange (29) + : : : +- ReusedExchange (32) + : : +- * Sort (41) + : : +- Exchange (40) + : : +- * Filter (39) + : : +- * ColumnarToRow (38) + : : +- Scan parquet default.store_returns (37) + : +- * Project (65) + : +- SortMergeJoin LeftOuter (64) + : :- * Sort (58) + : : +- Exchange (57) + : : +- * Project (56) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- * Project (53) + : : : +- * BroadcastHashJoin Inner BuildRight (52) + : : : :- * Filter (50) + : : : : +- * ColumnarToRow (49) + : : : : +- Scan parquet default.web_sales (48) + : : : +- ReusedExchange (51) + : : +- ReusedExchange (54) + : +- * Sort (63) + : +- Exchange (62) + : +- * Filter (61) + : +- * ColumnarToRow (60) + : +- Scan parquet default.web_returns (59) + +- * Sort (135) + +- Exchange (134) + +- * HashAggregate (133) + +- Exchange (132) + +- * HashAggregate (131) + +- * HashAggregate (130) + +- Exchange (129) + +- * HashAggregate (128) + +- Union (127) + :- * HashAggregate (111) + : +- Exchange (110) + : +- * HashAggregate (109) + : +- Union (108) + : :- * Project (92) + : : +- SortMergeJoin LeftOuter (91) + : : :- * Sort (88) + : : : +- Exchange (87) + : : : +- * Project (86) + : : : +- * BroadcastHashJoin Inner BuildRight (85) + : : : :- * Project (80) + : : : : +- * BroadcastHashJoin Inner BuildRight (79) + : : : : :- * Filter (77) + : : : : : +- * ColumnarToRow (76) + : : : : : +- Scan parquet default.catalog_sales (75) + : : : : +- ReusedExchange (78) + : : : +- BroadcastExchange (84) + : : : +- * Filter (83) + : : : +- * ColumnarToRow (82) + : : : +- Scan parquet default.date_dim (81) + : : +- * Sort (90) + : : +- ReusedExchange (89) + : +- * Project (107) + : +- SortMergeJoin LeftOuter (106) + : :- * Sort (103) + : : +- Exchange (102) + : : +- * Project (101) + : : +- * BroadcastHashJoin Inner BuildRight (100) + : : :- * Project (98) + : : : +- * BroadcastHashJoin Inner BuildRight (97) + : : : :- * Filter (95) + : : : : +- * ColumnarToRow (94) + : : : : +- Scan parquet default.store_sales (93) + : : : +- ReusedExchange (96) + : : +- ReusedExchange (99) + : +- * Sort (105) + : +- ReusedExchange (104) + +- * Project (126) + +- SortMergeJoin LeftOuter (125) + :- * Sort (122) + : +- Exchange (121) + : +- * Project (120) + : +- * BroadcastHashJoin Inner BuildRight (119) + : :- * Project (117) + : : +- * BroadcastHashJoin Inner BuildRight (116) + : : :- * Filter (114) + : : : +- * ColumnarToRow (113) + : : : +- Scan parquet default.web_sales (112) + : : +- ReusedExchange (115) + : +- ReusedExchange (118) + +- * Sort (124) + +- ReusedExchange (123) + + +(1) Scan parquet default.catalog_sales +Output [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 3] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] + +(3) Filter [codegen id : 3] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) + +(4) Scan parquet default.item +Output [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] + +(6) Filter [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] +Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_manufact_id#11)) + +(7) Project [codegen id : 1] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] + +(8) BroadcastExchange +Input [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] + +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(10) Project [codegen id : 3] +Output [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(11) Scan parquet default.date_dim +Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#13, d_year#14] + +(13) Filter [codegen id : 2] +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) + +(14) BroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(15) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(16) Project [codegen id : 3] +Output [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(17) Exchange +Input [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), true, [id=#16] + +(18) Sort [codegen id : 4] +Input [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST], false, 0 + +(19) Scan parquet default.catalog_returns +Output [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +ReadSchema: struct + +(20) ColumnarToRow [codegen id : 5] +Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] + +(21) Filter [codegen id : 5] +Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] +Condition : (isnotnull(cr_item_sk#17) AND isnotnull(cr_order_number#18)) + +(22) Exchange +Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] +Arguments: hashpartitioning(cr_order_number#18, cr_item_sk#17, 5), true, [id=#21] + +(23) Sort [codegen id : 6] +Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] +Arguments: [cr_order_number#18 ASC NULLS FIRST, cr_item_sk#17 ASC NULLS FIRST], false, 0 + +(24) SortMergeJoin +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#18, cr_item_sk#17] +Join condition: None + +(25) Project [codegen id : 7] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (cs_quantity#4 - coalesce(cr_return_quantity#19, 0)) AS sales_cnt#22, CheckOverflow((promote_precision(cast(cs_ext_sales_price#5 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#20, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#23] +Input [13]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] + +(26) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] +ReadSchema: struct + +(27) ColumnarToRow [codegen id : 10] +Input [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] + +(28) Filter [codegen id : 10] +Input [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] +Condition : (isnotnull(ss_item_sk#25) AND isnotnull(ss_sold_date_sk#24)) + +(29) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(30) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#25] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(31) Project [codegen id : 10] +Output [9]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(32) ReusedExchange [Reuses operator id: 14] +Output [2]: [d_date_sk#13, d_year#14] + +(33) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#24] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(34) Project [codegen id : 10] +Output [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(35) Exchange +Input [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: hashpartitioning(cast(ss_ticket_number#26 as bigint), cast(ss_item_sk#25 as bigint), 5), true, [id=#29] + +(36) Sort [codegen id : 11] +Input [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: [cast(ss_ticket_number#26 as bigint) ASC NULLS FIRST, cast(ss_item_sk#25 as bigint) ASC NULLS FIRST], false, 0 + +(37) Scan parquet default.store_returns +Output [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct + +(38) ColumnarToRow [codegen id : 12] +Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] + +(39) Filter [codegen id : 12] +Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] +Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) + +(40) Exchange +Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#34] + +(41) Sort [codegen id : 13] +Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] +Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 + +(42) SortMergeJoin +Left keys [2]: [cast(ss_ticket_number#26 as bigint), cast(ss_item_sk#25 as bigint)] +Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] +Join condition: None + +(43) Project [codegen id : 14] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (ss_quantity#27 - coalesce(sr_return_quantity#32, 0)) AS sales_cnt#35, CheckOverflow((promote_precision(cast(ss_ext_sales_price#28 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#33, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#36] +Input [13]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] + +(44) Union + +(45) HashAggregate [codegen id : 15] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] + +(46) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23, 5), true, [id=#37] + +(47) HashAggregate [codegen id : 16] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] + +(48) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] +ReadSchema: struct + +(49) ColumnarToRow [codegen id : 19] +Input [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] + +(50) Filter [codegen id : 19] +Input [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] +Condition : (isnotnull(ws_item_sk#39) AND isnotnull(ws_sold_date_sk#38)) + +(51) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(52) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(53) Project [codegen id : 19] +Output [9]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(54) ReusedExchange [Reuses operator id: 14] +Output [2]: [d_date_sk#13, d_year#14] + +(55) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(56) Project [codegen id : 19] +Output [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(57) Exchange +Input [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: hashpartitioning(cast(ws_order_number#40 as bigint), cast(ws_item_sk#39 as bigint), 5), true, [id=#43] + +(58) Sort [codegen id : 20] +Input [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: [cast(ws_order_number#40 as bigint) ASC NULLS FIRST, cast(ws_item_sk#39 as bigint) ASC NULLS FIRST], false, 0 + +(59) Scan parquet default.web_returns +Output [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] +ReadSchema: struct + +(60) ColumnarToRow [codegen id : 21] +Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] + +(61) Filter [codegen id : 21] +Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] +Condition : (isnotnull(wr_order_number#45) AND isnotnull(wr_item_sk#44)) + +(62) Exchange +Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] +Arguments: hashpartitioning(wr_order_number#45, wr_item_sk#44, 5), true, [id=#48] + +(63) Sort [codegen id : 22] +Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] +Arguments: [wr_order_number#45 ASC NULLS FIRST, wr_item_sk#44 ASC NULLS FIRST], false, 0 + +(64) SortMergeJoin +Left keys [2]: [cast(ws_order_number#40 as bigint), cast(ws_item_sk#39 as bigint)] +Right keys [2]: [wr_order_number#45, wr_item_sk#44] +Join condition: None + +(65) Project [codegen id : 23] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (ws_quantity#41 - coalesce(wr_return_quantity#46, 0)) AS sales_cnt#49, CheckOverflow((promote_precision(cast(ws_ext_sales_price#42 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#47, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#50] +Input [13]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] + +(66) Union + +(67) HashAggregate [codegen id : 24] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] + +(68) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23, 5), true, [id=#51] + +(69) HashAggregate [codegen id : 25] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] + +(70) HashAggregate [codegen id : 25] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Keys [5]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Functions [2]: [partial_sum(cast(sales_cnt#22 as bigint)), partial_sum(UnscaledValue(sales_amt#23))] +Aggregate Attributes [2]: [sum#52, sum#53] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#54, sum#55] + +(71) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#54, sum#55] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, 5), true, [id=#56] + +(72) HashAggregate [codegen id : 26] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#54, sum#55] +Keys [5]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Functions [2]: [sum(cast(sales_cnt#22 as bigint)), sum(UnscaledValue(sales_amt#23))] +Aggregate Attributes [2]: [sum(cast(sales_cnt#22 as bigint))#57, sum(UnscaledValue(sales_amt#23))#58] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum(cast(sales_cnt#22 as bigint))#57 AS sales_cnt#59, MakeDecimal(sum(UnscaledValue(sales_amt#23))#58,18,2) AS sales_amt#60] + +(73) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#59, sales_amt#60] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, 5), true, [id=#61] + +(74) Sort [codegen id : 27] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#59, sales_amt#60] +Arguments: [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_manufact_id#11 ASC NULLS FIRST], false, 0 + +(75) Scan parquet default.catalog_sales +Output [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] +ReadSchema: struct + +(76) ColumnarToRow [codegen id : 30] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] + +(77) Filter [codegen id : 30] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) + +(78) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(79) BroadcastHashJoin [codegen id : 30] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#62] +Join condition: None + +(80) Project [codegen id : 30] +Output [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Input [10]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(81) Scan parquet default.date_dim +Output [2]: [d_date_sk#67, d_year#68] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(82) ColumnarToRow [codegen id : 29] +Input [2]: [d_date_sk#67, d_year#68] + +(83) Filter [codegen id : 29] +Input [2]: [d_date_sk#67, d_year#68] +Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2001)) AND isnotnull(d_date_sk#67)) + +(84) BroadcastExchange +Input [2]: [d_date_sk#67, d_year#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] + +(85) BroadcastHashJoin [codegen id : 30] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#67] +Join condition: None + +(86) Project [codegen id : 30] +Output [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_date_sk#67, d_year#68] + +(87) Exchange +Input [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), true, [id=#70] + +(88) Sort [codegen id : 31] +Input [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST], false, 0 + +(89) ReusedExchange [Reuses operator id: 22] +Output [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] + +(90) Sort [codegen id : 33] +Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] +Arguments: [cr_order_number#18 ASC NULLS FIRST, cr_item_sk#17 ASC NULLS FIRST], false, 0 + +(91) SortMergeJoin +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#18, cr_item_sk#17] +Join condition: None + +(92) Project [codegen id : 34] +Output [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, (cs_quantity#4 - coalesce(cr_return_quantity#19, 0)) AS sales_cnt#22, CheckOverflow((promote_precision(cast(cs_ext_sales_price#5 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#20, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#23] +Input [13]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68, cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] + +(93) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] +ReadSchema: struct + +(94) ColumnarToRow [codegen id : 37] +Input [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] + +(95) Filter [codegen id : 37] +Input [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] +Condition : (isnotnull(ss_item_sk#25) AND isnotnull(ss_sold_date_sk#24)) + +(96) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(97) BroadcastHashJoin [codegen id : 37] +Left keys [1]: [ss_item_sk#25] +Right keys [1]: [i_item_sk#62] +Join condition: None + +(98) Project [codegen id : 37] +Output [9]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Input [10]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(99) ReusedExchange [Reuses operator id: 84] +Output [2]: [d_date_sk#67, d_year#68] + +(100) BroadcastHashJoin [codegen id : 37] +Left keys [1]: [ss_sold_date_sk#24] +Right keys [1]: [d_date_sk#67] +Join condition: None + +(101) Project [codegen id : 37] +Output [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Input [11]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_date_sk#67, d_year#68] + +(102) Exchange +Input [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: hashpartitioning(cast(ss_ticket_number#26 as bigint), cast(ss_item_sk#25 as bigint), 5), true, [id=#71] + +(103) Sort [codegen id : 38] +Input [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: [cast(ss_ticket_number#26 as bigint) ASC NULLS FIRST, cast(ss_item_sk#25 as bigint) ASC NULLS FIRST], false, 0 + +(104) ReusedExchange [Reuses operator id: 40] +Output [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] + +(105) Sort [codegen id : 40] +Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] +Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 + +(106) SortMergeJoin +Left keys [2]: [cast(ss_ticket_number#26 as bigint), cast(ss_item_sk#25 as bigint)] +Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] +Join condition: None + +(107) Project [codegen id : 41] +Output [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, (ss_quantity#27 - coalesce(sr_return_quantity#32, 0)) AS sales_cnt#72, CheckOverflow((promote_precision(cast(ss_ext_sales_price#28 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#33, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#73] +Input [13]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68, sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] + +(108) Union + +(109) HashAggregate [codegen id : 42] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] + +(110) Exchange +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Arguments: hashpartitioning(d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23, 5), true, [id=#74] + +(111) HashAggregate [codegen id : 43] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] + +(112) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] +ReadSchema: struct + +(113) ColumnarToRow [codegen id : 46] +Input [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] + +(114) Filter [codegen id : 46] +Input [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] +Condition : (isnotnull(ws_item_sk#39) AND isnotnull(ws_sold_date_sk#38)) + +(115) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(116) BroadcastHashJoin [codegen id : 46] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#62] +Join condition: None + +(117) Project [codegen id : 46] +Output [9]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Input [10]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(118) ReusedExchange [Reuses operator id: 84] +Output [2]: [d_date_sk#67, d_year#68] + +(119) BroadcastHashJoin [codegen id : 46] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#67] +Join condition: None + +(120) Project [codegen id : 46] +Output [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Input [11]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_date_sk#67, d_year#68] + +(121) Exchange +Input [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: hashpartitioning(cast(ws_order_number#40 as bigint), cast(ws_item_sk#39 as bigint), 5), true, [id=#75] + +(122) Sort [codegen id : 47] +Input [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: [cast(ws_order_number#40 as bigint) ASC NULLS FIRST, cast(ws_item_sk#39 as bigint) ASC NULLS FIRST], false, 0 + +(123) ReusedExchange [Reuses operator id: 62] +Output [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] + +(124) Sort [codegen id : 49] +Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] +Arguments: [wr_order_number#45 ASC NULLS FIRST, wr_item_sk#44 ASC NULLS FIRST], false, 0 + +(125) SortMergeJoin +Left keys [2]: [cast(ws_order_number#40 as bigint), cast(ws_item_sk#39 as bigint)] +Right keys [2]: [wr_order_number#45, wr_item_sk#44] +Join condition: None + +(126) Project [codegen id : 50] +Output [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, (ws_quantity#41 - coalesce(wr_return_quantity#46, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ws_ext_sales_price#42 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#47, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] +Input [13]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68, wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] + +(127) Union + +(128) HashAggregate [codegen id : 51] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] + +(129) Exchange +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Arguments: hashpartitioning(d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23, 5), true, [id=#78] + +(130) HashAggregate [codegen id : 52] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] + +(131) HashAggregate [codegen id : 52] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Keys [5]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Functions [2]: [partial_sum(cast(sales_cnt#22 as bigint)), partial_sum(UnscaledValue(sales_amt#23))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sum#81, sum#82] + +(132) Exchange +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sum#81, sum#82] +Arguments: hashpartitioning(d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, 5), true, [id=#83] + +(133) HashAggregate [codegen id : 53] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sum#81, sum#82] +Keys [5]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Functions [2]: [sum(cast(sales_cnt#22 as bigint)), sum(UnscaledValue(sales_amt#23))] +Aggregate Attributes [2]: [sum(cast(sales_cnt#22 as bigint))#84, sum(UnscaledValue(sales_amt#23))#85] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sum(cast(sales_cnt#22 as bigint))#84 AS sales_cnt#86, MakeDecimal(sum(UnscaledValue(sales_amt#23))#85,18,2) AS sales_amt#87] + +(134) Exchange +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#86, sales_amt#87] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, 5), true, [id=#88] + +(135) Sort [codegen id : 54] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#86, sales_amt#87] +Arguments: [i_brand_id#63 ASC NULLS FIRST, i_class_id#64 ASC NULLS FIRST, i_category_id#65 ASC NULLS FIRST, i_manufact_id#66 ASC NULLS FIRST], false, 0 + +(136) SortMergeJoin [codegen id : 55] +Left keys [4]: [i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Right keys [4]: [i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#59 as decimal(17,2))) / promote_precision(cast(sales_cnt#86 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) + +(137) Project [codegen id : 55] +Output [10]: [d_year#68 AS prev_year#89, d_year#14 AS year#90, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#86 AS prev_yr_cnt#91, sales_cnt#59 AS curr_yr_cnt#92, (sales_cnt#59 - sales_cnt#86) AS sales_cnt_diff#93, CheckOverflow((promote_precision(cast(sales_amt#60 as decimal(19,2))) - promote_precision(cast(sales_amt#87 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#94] +Input [14]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#59, sales_amt#60, d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#86, sales_amt#87] + +(138) TakeOrderedAndProject +Input [10]: [prev_year#89, year#90, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, prev_yr_cnt#91, curr_yr_cnt#92, sales_cnt_diff#93, sales_amt_diff#94] +Arguments: 100, [sales_cnt_diff#93 ASC NULLS FIRST], [prev_year#89, year#90, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, prev_yr_cnt#91, curr_yr_cnt#92, sales_cnt_diff#93, sales_amt_diff#94] + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/SPARK-33152_simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/SPARK-33152_simplified.txt new file mode 100644 index 000000000000..3a7d2419a50a --- /dev/null +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/SPARK-33152_simplified.txt @@ -0,0 +1,237 @@ +TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] + WholeStageCodegen (55) + Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] + InputAdapter + WholeStageCodegen (27) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + WholeStageCodegen (26) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + WholeStageCodegen (25) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + WholeStageCodegen (24) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (16) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #4 + WholeStageCodegen (15) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (7) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + InputAdapter + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + WholeStageCodegen (4) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #5 + WholeStageCodegen (3) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (6) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #8 + WholeStageCodegen (5) + Filter [cr_item_sk,cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + WholeStageCodegen (14) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + InputAdapter + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + WholeStageCodegen (11) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #9 + WholeStageCodegen (10) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + WholeStageCodegen (13) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #10 + WholeStageCodegen (12) + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + WholeStageCodegen (23) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + InputAdapter + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + WholeStageCodegen (20) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #11 + WholeStageCodegen (19) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + WholeStageCodegen (22) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #12 + WholeStageCodegen (21) + Filter [wr_order_number,wr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + InputAdapter + WholeStageCodegen (54) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + WholeStageCodegen (53) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + WholeStageCodegen (52) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + WholeStageCodegen (51) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (43) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #16 + WholeStageCodegen (42) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (34) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + InputAdapter + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + WholeStageCodegen (31) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #17 + WholeStageCodegen (30) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + BroadcastExchange #18 + WholeStageCodegen (29) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (33) + Sort [cr_order_number,cr_item_sk] + InputAdapter + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + WholeStageCodegen (41) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + InputAdapter + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + WholeStageCodegen (38) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #19 + WholeStageCodegen (37) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #18 + WholeStageCodegen (40) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + WholeStageCodegen (50) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + InputAdapter + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + WholeStageCodegen (47) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #20 + WholeStageCodegen (46) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #18 + WholeStageCodegen (49) + Sort [wr_order_number,wr_item_sk] + InputAdapter + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/SPARK-33152_explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/SPARK-33152_explain.txt new file mode 100644 index 000000000000..a4305fe896a6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/SPARK-33152_explain.txt @@ -0,0 +1,647 @@ +== Physical Plan == +TakeOrderedAndProject (117) ++- * Project (116) + +- * BroadcastHashJoin Inner BuildRight (115) + :- * HashAggregate (63) + : +- Exchange (62) + : +- * HashAggregate (61) + : +- * HashAggregate (60) + : +- Exchange (59) + : +- * HashAggregate (58) + : +- Union (57) + : :- * HashAggregate (41) + : : +- Exchange (40) + : : +- * HashAggregate (39) + : : +- Union (38) + : : :- * Project (22) + : : : +- * BroadcastHashJoin LeftOuter BuildRight (21) + : : : :- * Project (16) + : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : :- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * Filter (3) + : : : : : : +- * ColumnarToRow (2) + : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * Project (7) + : : : : : +- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.item (4) + : : : : +- BroadcastExchange (14) + : : : : +- * Filter (13) + : : : : +- * ColumnarToRow (12) + : : : : +- Scan parquet default.date_dim (11) + : : : +- BroadcastExchange (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_returns (17) + : : +- * Project (37) + : : +- * BroadcastHashJoin LeftOuter BuildRight (36) + : : :- * Project (31) + : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : :- * Project (28) + : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : :- * Filter (25) + : : : : : +- * ColumnarToRow (24) + : : : : : +- Scan parquet default.store_sales (23) + : : : : +- ReusedExchange (26) + : : : +- ReusedExchange (29) + : : +- BroadcastExchange (35) + : : +- * Filter (34) + : : +- * ColumnarToRow (33) + : : +- Scan parquet default.store_returns (32) + : +- * Project (56) + : +- * BroadcastHashJoin LeftOuter BuildRight (55) + : :- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * Project (47) + : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : :- * Filter (44) + : : : : +- * ColumnarToRow (43) + : : : : +- Scan parquet default.web_sales (42) + : : : +- ReusedExchange (45) + : : +- ReusedExchange (48) + : +- BroadcastExchange (54) + : +- * Filter (53) + : +- * ColumnarToRow (52) + : +- Scan parquet default.web_returns (51) + +- BroadcastExchange (114) + +- * HashAggregate (113) + +- Exchange (112) + +- * HashAggregate (111) + +- * HashAggregate (110) + +- Exchange (109) + +- * HashAggregate (108) + +- Union (107) + :- * HashAggregate (94) + : +- Exchange (93) + : +- * HashAggregate (92) + : +- Union (91) + : :- * Project (78) + : : +- * BroadcastHashJoin LeftOuter BuildRight (77) + : : :- * Project (75) + : : : +- * BroadcastHashJoin Inner BuildRight (74) + : : : :- * Project (69) + : : : : +- * BroadcastHashJoin Inner BuildRight (68) + : : : : :- * Filter (66) + : : : : : +- * ColumnarToRow (65) + : : : : : +- Scan parquet default.catalog_sales (64) + : : : : +- ReusedExchange (67) + : : : +- BroadcastExchange (73) + : : : +- * Filter (72) + : : : +- * ColumnarToRow (71) + : : : +- Scan parquet default.date_dim (70) + : : +- ReusedExchange (76) + : +- * Project (90) + : +- * BroadcastHashJoin LeftOuter BuildRight (89) + : :- * Project (87) + : : +- * BroadcastHashJoin Inner BuildRight (86) + : : :- * Project (84) + : : : +- * BroadcastHashJoin Inner BuildRight (83) + : : : :- * Filter (81) + : : : : +- * ColumnarToRow (80) + : : : : +- Scan parquet default.store_sales (79) + : : : +- ReusedExchange (82) + : : +- ReusedExchange (85) + : +- ReusedExchange (88) + +- * Project (106) + +- * BroadcastHashJoin LeftOuter BuildRight (105) + :- * Project (103) + : +- * BroadcastHashJoin Inner BuildRight (102) + : :- * Project (100) + : : +- * BroadcastHashJoin Inner BuildRight (99) + : : :- * Filter (97) + : : : +- * ColumnarToRow (96) + : : : +- Scan parquet default.web_sales (95) + : : +- ReusedExchange (98) + : +- ReusedExchange (101) + +- ReusedExchange (104) + + +(1) Scan parquet default.catalog_sales +Output [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 4] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] + +(3) Filter [codegen id : 4] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) + +(4) Scan parquet default.item +Output [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] + +(6) Filter [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] +Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_manufact_id#11)) + +(7) Project [codegen id : 1] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] + +(8) BroadcastExchange +Input [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] + +(9) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(10) Project [codegen id : 4] +Output [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(11) Scan parquet default.date_dim +Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#13, d_year#14] + +(13) Filter [codegen id : 2] +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) + +(14) BroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(15) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(16) Project [codegen id : 4] +Output [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(17) Scan parquet default.catalog_returns +Output [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +ReadSchema: struct + +(18) ColumnarToRow [codegen id : 3] +Input [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] + +(19) Filter [codegen id : 3] +Input [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] +Condition : (isnotnull(cr_item_sk#16) AND isnotnull(cr_order_number#17)) + +(20) BroadcastExchange +Input [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [id=#20] + +(21) BroadcastHashJoin [codegen id : 4] +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#17, cr_item_sk#16] +Join condition: None + +(22) Project [codegen id : 4] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (cs_quantity#4 - coalesce(cr_return_quantity#18, 0)) AS sales_cnt#21, CheckOverflow((promote_precision(cast(cs_ext_sales_price#5 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#19, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#22] +Input [13]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] + +(23) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] +ReadSchema: struct + +(24) ColumnarToRow [codegen id : 8] +Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] + +(25) Filter [codegen id : 8] +Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] +Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_date_sk#23)) + +(26) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(27) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_item_sk#24] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(28) Project [codegen id : 8] +Output [9]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(29) ReusedExchange [Reuses operator id: 14] +Output [2]: [d_date_sk#13, d_year#14] + +(30) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(31) Project [codegen id : 8] +Output [9]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(32) Scan parquet default.store_returns +Output [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct + +(33) ColumnarToRow [codegen id : 7] +Input [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] + +(34) Filter [codegen id : 7] +Input [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] +Condition : (isnotnull(sr_ticket_number#29) AND isnotnull(sr_item_sk#28)) + +(35) BroadcastExchange +Input [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#32] + +(36) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [cast(ss_ticket_number#25 as bigint), cast(ss_item_sk#24 as bigint)] +Right keys [2]: [sr_ticket_number#29, sr_item_sk#28] +Join condition: None + +(37) Project [codegen id : 8] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (ss_quantity#26 - coalesce(sr_return_quantity#30, 0)) AS sales_cnt#33, CheckOverflow((promote_precision(cast(ss_ext_sales_price#27 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#31, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#34] +Input [13]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] + +(38) Union + +(39) HashAggregate [codegen id : 9] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] + +(40) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22, 5), true, [id=#35] + +(41) HashAggregate [codegen id : 10] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] + +(42) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 14] +Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] + +(44) Filter [codegen id : 14] +Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] +Condition : (isnotnull(ws_item_sk#37) AND isnotnull(ws_sold_date_sk#36)) + +(45) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(46) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(47) Project [codegen id : 14] +Output [9]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(48) ReusedExchange [Reuses operator id: 14] +Output [2]: [d_date_sk#13, d_year#14] + +(49) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(50) Project [codegen id : 14] +Output [9]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(51) Scan parquet default.web_returns +Output [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] +ReadSchema: struct + +(52) ColumnarToRow [codegen id : 13] +Input [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] + +(53) Filter [codegen id : 13] +Input [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] +Condition : (isnotnull(wr_order_number#42) AND isnotnull(wr_item_sk#41)) + +(54) BroadcastExchange +Input [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#45] + +(55) BroadcastHashJoin [codegen id : 14] +Left keys [2]: [cast(ws_order_number#38 as bigint), cast(ws_item_sk#37 as bigint)] +Right keys [2]: [wr_order_number#42, wr_item_sk#41] +Join condition: None + +(56) Project [codegen id : 14] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (ws_quantity#39 - coalesce(wr_return_quantity#43, 0)) AS sales_cnt#46, CheckOverflow((promote_precision(cast(ws_ext_sales_price#40 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#44, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#47] +Input [13]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] + +(57) Union + +(58) HashAggregate [codegen id : 15] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] + +(59) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22, 5), true, [id=#48] + +(60) HashAggregate [codegen id : 16] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] + +(61) HashAggregate [codegen id : 16] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Keys [5]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Functions [2]: [partial_sum(cast(sales_cnt#21 as bigint)), partial_sum(UnscaledValue(sales_amt#22))] +Aggregate Attributes [2]: [sum#49, sum#50] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#51, sum#52] + +(62) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#51, sum#52] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, 5), true, [id=#53] + +(63) HashAggregate [codegen id : 34] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#51, sum#52] +Keys [5]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Functions [2]: [sum(cast(sales_cnt#21 as bigint)), sum(UnscaledValue(sales_amt#22))] +Aggregate Attributes [2]: [sum(cast(sales_cnt#21 as bigint))#54, sum(UnscaledValue(sales_amt#22))#55] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum(cast(sales_cnt#21 as bigint))#54 AS sales_cnt#56, MakeDecimal(sum(UnscaledValue(sales_amt#22))#55,18,2) AS sales_amt#57] + +(64) Scan parquet default.catalog_sales +Output [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] +ReadSchema: struct + +(65) ColumnarToRow [codegen id : 20] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] + +(66) Filter [codegen id : 20] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) + +(67) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(68) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#58] +Join condition: None + +(69) Project [codegen id : 20] +Output [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Input [10]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(70) Scan parquet default.date_dim +Output [2]: [d_date_sk#63, d_year#64] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(71) ColumnarToRow [codegen id : 18] +Input [2]: [d_date_sk#63, d_year#64] + +(72) Filter [codegen id : 18] +Input [2]: [d_date_sk#63, d_year#64] +Condition : ((isnotnull(d_year#64) AND (d_year#64 = 2001)) AND isnotnull(d_date_sk#63)) + +(73) BroadcastExchange +Input [2]: [d_date_sk#63, d_year#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] + +(74) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#63] +Join condition: None + +(75) Project [codegen id : 20] +Output [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_date_sk#63, d_year#64] + +(76) ReusedExchange [Reuses operator id: 20] +Output [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] + +(77) BroadcastHashJoin [codegen id : 20] +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#17, cr_item_sk#16] +Join condition: None + +(78) Project [codegen id : 20] +Output [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, (cs_quantity#4 - coalesce(cr_return_quantity#18, 0)) AS sales_cnt#21, CheckOverflow((promote_precision(cast(cs_ext_sales_price#5 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#19, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#22] +Input [13]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64, cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] + +(79) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] +ReadSchema: struct + +(80) ColumnarToRow [codegen id : 24] +Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] + +(81) Filter [codegen id : 24] +Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] +Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_date_sk#23)) + +(82) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(83) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_item_sk#24] +Right keys [1]: [i_item_sk#58] +Join condition: None + +(84) Project [codegen id : 24] +Output [9]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Input [10]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(85) ReusedExchange [Reuses operator id: 73] +Output [2]: [d_date_sk#63, d_year#64] + +(86) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#63] +Join condition: None + +(87) Project [codegen id : 24] +Output [9]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64] +Input [11]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_date_sk#63, d_year#64] + +(88) ReusedExchange [Reuses operator id: 35] +Output [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] + +(89) BroadcastHashJoin [codegen id : 24] +Left keys [2]: [cast(ss_ticket_number#25 as bigint), cast(ss_item_sk#24 as bigint)] +Right keys [2]: [sr_ticket_number#29, sr_item_sk#28] +Join condition: None + +(90) Project [codegen id : 24] +Output [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, (ss_quantity#26 - coalesce(sr_return_quantity#30, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ss_ext_sales_price#27 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#31, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64, sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] + +(91) Union + +(92) HashAggregate [codegen id : 25] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] + +(93) Exchange +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Arguments: hashpartitioning(d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22, 5), true, [id=#68] + +(94) HashAggregate [codegen id : 26] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] + +(95) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] +ReadSchema: struct + +(96) ColumnarToRow [codegen id : 30] +Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] + +(97) Filter [codegen id : 30] +Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] +Condition : (isnotnull(ws_item_sk#37) AND isnotnull(ws_sold_date_sk#36)) + +(98) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(99) BroadcastHashJoin [codegen id : 30] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#58] +Join condition: None + +(100) Project [codegen id : 30] +Output [9]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Input [10]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(101) ReusedExchange [Reuses operator id: 73] +Output [2]: [d_date_sk#63, d_year#64] + +(102) BroadcastHashJoin [codegen id : 30] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#63] +Join condition: None + +(103) Project [codegen id : 30] +Output [9]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64] +Input [11]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_date_sk#63, d_year#64] + +(104) ReusedExchange [Reuses operator id: 54] +Output [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] + +(105) BroadcastHashJoin [codegen id : 30] +Left keys [2]: [cast(ws_order_number#38 as bigint), cast(ws_item_sk#37 as bigint)] +Right keys [2]: [wr_order_number#42, wr_item_sk#41] +Join condition: None + +(106) Project [codegen id : 30] +Output [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, (ws_quantity#39 - coalesce(wr_return_quantity#43, 0)) AS sales_cnt#69, CheckOverflow((promote_precision(cast(ws_ext_sales_price#40 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#44, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#70] +Input [13]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64, wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] + +(107) Union + +(108) HashAggregate [codegen id : 31] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] + +(109) Exchange +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Arguments: hashpartitioning(d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22, 5), true, [id=#71] + +(110) HashAggregate [codegen id : 32] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] + +(111) HashAggregate [codegen id : 32] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Keys [5]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Functions [2]: [partial_sum(cast(sales_cnt#21 as bigint)), partial_sum(UnscaledValue(sales_amt#22))] +Aggregate Attributes [2]: [sum#72, sum#73] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum#74, sum#75] + +(112) Exchange +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum#74, sum#75] +Arguments: hashpartitioning(d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, 5), true, [id=#76] + +(113) HashAggregate [codegen id : 33] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum#74, sum#75] +Keys [5]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Functions [2]: [sum(cast(sales_cnt#21 as bigint)), sum(UnscaledValue(sales_amt#22))] +Aggregate Attributes [2]: [sum(cast(sales_cnt#21 as bigint))#77, sum(UnscaledValue(sales_amt#22))#78] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum(cast(sales_cnt#21 as bigint))#77 AS sales_cnt#79, MakeDecimal(sum(UnscaledValue(sales_amt#22))#78,18,2) AS sales_amt#80] + +(114) BroadcastExchange +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#79, sales_amt#80] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true], input[4, int, true]),false), [id=#81] + +(115) BroadcastHashJoin [codegen id : 34] +Left keys [4]: [i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Right keys [4]: [i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#56 as decimal(17,2))) / promote_precision(cast(sales_cnt#79 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) + +(116) Project [codegen id : 34] +Output [10]: [d_year#64 AS prev_year#82, d_year#14 AS year#83, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#79 AS prev_yr_cnt#84, sales_cnt#56 AS curr_yr_cnt#85, (sales_cnt#56 - sales_cnt#79) AS sales_cnt_diff#86, CheckOverflow((promote_precision(cast(sales_amt#57 as decimal(19,2))) - promote_precision(cast(sales_amt#80 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#87] +Input [14]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#56, sales_amt#57, d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#79, sales_amt#80] + +(117) TakeOrderedAndProject +Input [10]: [prev_year#82, year#83, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, prev_yr_cnt#84, curr_yr_cnt#85, sales_cnt_diff#86, sales_amt_diff#87] +Arguments: 100, [sales_cnt_diff#86 ASC NULLS FIRST], [prev_year#82, year#83, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, prev_yr_cnt#84, curr_yr_cnt#85, sales_cnt_diff#86, sales_amt_diff#87] + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/SPARK-33152_simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/SPARK-33152_simplified.txt new file mode 100644 index 000000000000..9cc750d4710e --- /dev/null +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/SPARK-33152_simplified.txt @@ -0,0 +1,180 @@ +TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] + WholeStageCodegen (34) + Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + WholeStageCodegen (16) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #2 + WholeStageCodegen (15) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (10) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + WholeStageCodegen (9) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (4) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Filter [cr_item_sk,cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + WholeStageCodegen (8) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + WholeStageCodegen (14) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (13) + Filter [wr_order_number,wr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (33) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #10 + WholeStageCodegen (32) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #11 + WholeStageCodegen (31) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (26) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #12 + WholeStageCodegen (25) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (20) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (18) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #6 + WholeStageCodegen (24) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year] #13 + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #7 + WholeStageCodegen (30) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year] #13 + InputAdapter + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/SPARK-33152_explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/SPARK-33152_explain.txt new file mode 100644 index 000000000000..eff5044ef033 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/SPARK-33152_explain.txt @@ -0,0 +1,752 @@ +== Physical Plan == +TakeOrderedAndProject (138) ++- * Project (137) + +- * SortMergeJoin Inner (136) + :- * Sort (74) + : +- Exchange (73) + : +- * HashAggregate (72) + : +- Exchange (71) + : +- * HashAggregate (70) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- Union (66) + : :- * HashAggregate (47) + : : +- Exchange (46) + : : +- * HashAggregate (45) + : : +- Union (44) + : : :- * Project (25) + : : : +- SortMergeJoin LeftOuter (24) + : : : :- * Sort (18) + : : : : +- Exchange (17) + : : : : +- * Project (16) + : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : :- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * Filter (3) + : : : : : : +- * ColumnarToRow (2) + : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * Project (7) + : : : : : +- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.item (4) + : : : : +- BroadcastExchange (14) + : : : : +- * Filter (13) + : : : : +- * ColumnarToRow (12) + : : : : +- Scan parquet default.date_dim (11) + : : : +- * Sort (23) + : : : +- Exchange (22) + : : : +- * Filter (21) + : : : +- * ColumnarToRow (20) + : : : +- Scan parquet default.catalog_returns (19) + : : +- * Project (43) + : : +- SortMergeJoin LeftOuter (42) + : : :- * Sort (36) + : : : +- Exchange (35) + : : : +- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * Project (31) + : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : :- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.store_sales (26) + : : : : +- ReusedExchange (29) + : : : +- ReusedExchange (32) + : : +- * Sort (41) + : : +- Exchange (40) + : : +- * Filter (39) + : : +- * ColumnarToRow (38) + : : +- Scan parquet default.store_returns (37) + : +- * Project (65) + : +- SortMergeJoin LeftOuter (64) + : :- * Sort (58) + : : +- Exchange (57) + : : +- * Project (56) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- * Project (53) + : : : +- * BroadcastHashJoin Inner BuildRight (52) + : : : :- * Filter (50) + : : : : +- * ColumnarToRow (49) + : : : : +- Scan parquet default.web_sales (48) + : : : +- ReusedExchange (51) + : : +- ReusedExchange (54) + : +- * Sort (63) + : +- Exchange (62) + : +- * Filter (61) + : +- * ColumnarToRow (60) + : +- Scan parquet default.web_returns (59) + +- * Sort (135) + +- Exchange (134) + +- * HashAggregate (133) + +- Exchange (132) + +- * HashAggregate (131) + +- * HashAggregate (130) + +- Exchange (129) + +- * HashAggregate (128) + +- Union (127) + :- * HashAggregate (111) + : +- Exchange (110) + : +- * HashAggregate (109) + : +- Union (108) + : :- * Project (92) + : : +- SortMergeJoin LeftOuter (91) + : : :- * Sort (88) + : : : +- Exchange (87) + : : : +- * Project (86) + : : : +- * BroadcastHashJoin Inner BuildRight (85) + : : : :- * Project (80) + : : : : +- * BroadcastHashJoin Inner BuildRight (79) + : : : : :- * Filter (77) + : : : : : +- * ColumnarToRow (76) + : : : : : +- Scan parquet default.catalog_sales (75) + : : : : +- ReusedExchange (78) + : : : +- BroadcastExchange (84) + : : : +- * Filter (83) + : : : +- * ColumnarToRow (82) + : : : +- Scan parquet default.date_dim (81) + : : +- * Sort (90) + : : +- ReusedExchange (89) + : +- * Project (107) + : +- SortMergeJoin LeftOuter (106) + : :- * Sort (103) + : : +- Exchange (102) + : : +- * Project (101) + : : +- * BroadcastHashJoin Inner BuildRight (100) + : : :- * Project (98) + : : : +- * BroadcastHashJoin Inner BuildRight (97) + : : : :- * Filter (95) + : : : : +- * ColumnarToRow (94) + : : : : +- Scan parquet default.store_sales (93) + : : : +- ReusedExchange (96) + : : +- ReusedExchange (99) + : +- * Sort (105) + : +- ReusedExchange (104) + +- * Project (126) + +- SortMergeJoin LeftOuter (125) + :- * Sort (122) + : +- Exchange (121) + : +- * Project (120) + : +- * BroadcastHashJoin Inner BuildRight (119) + : :- * Project (117) + : : +- * BroadcastHashJoin Inner BuildRight (116) + : : :- * Filter (114) + : : : +- * ColumnarToRow (113) + : : : +- Scan parquet default.web_sales (112) + : : +- ReusedExchange (115) + : +- ReusedExchange (118) + +- * Sort (124) + +- ReusedExchange (123) + + +(1) Scan parquet default.catalog_sales +Output [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 3] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] + +(3) Filter [codegen id : 3] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) + +(4) Scan parquet default.item +Output [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] + +(6) Filter [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] +Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_manufact_id#11)) + +(7) Project [codegen id : 1] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] + +(8) BroadcastExchange +Input [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] + +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(10) Project [codegen id : 3] +Output [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(11) Scan parquet default.date_dim +Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#13, d_year#14] + +(13) Filter [codegen id : 2] +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) + +(14) BroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(15) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(16) Project [codegen id : 3] +Output [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(17) Exchange +Input [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), true, [id=#16] + +(18) Sort [codegen id : 4] +Input [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST], false, 0 + +(19) Scan parquet default.catalog_returns +Output [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +ReadSchema: struct + +(20) ColumnarToRow [codegen id : 5] +Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] + +(21) Filter [codegen id : 5] +Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] +Condition : (isnotnull(cr_item_sk#17) AND isnotnull(cr_order_number#18)) + +(22) Exchange +Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] +Arguments: hashpartitioning(cr_order_number#18, cr_item_sk#17, 5), true, [id=#21] + +(23) Sort [codegen id : 6] +Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] +Arguments: [cr_order_number#18 ASC NULLS FIRST, cr_item_sk#17 ASC NULLS FIRST], false, 0 + +(24) SortMergeJoin +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#18, cr_item_sk#17] +Join condition: None + +(25) Project [codegen id : 7] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (cs_quantity#4 - coalesce(cr_return_quantity#19, 0)) AS sales_cnt#22, CheckOverflow((promote_precision(cast(cs_ext_sales_price#5 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#20, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#23] +Input [13]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] + +(26) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] +ReadSchema: struct + +(27) ColumnarToRow [codegen id : 10] +Input [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] + +(28) Filter [codegen id : 10] +Input [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] +Condition : (isnotnull(ss_item_sk#25) AND isnotnull(ss_sold_date_sk#24)) + +(29) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(30) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#25] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(31) Project [codegen id : 10] +Output [9]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(32) ReusedExchange [Reuses operator id: 14] +Output [2]: [d_date_sk#13, d_year#14] + +(33) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#24] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(34) Project [codegen id : 10] +Output [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(35) Exchange +Input [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: hashpartitioning(cast(ss_ticket_number#26 as bigint), cast(ss_item_sk#25 as bigint), 5), true, [id=#29] + +(36) Sort [codegen id : 11] +Input [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: [cast(ss_ticket_number#26 as bigint) ASC NULLS FIRST, cast(ss_item_sk#25 as bigint) ASC NULLS FIRST], false, 0 + +(37) Scan parquet default.store_returns +Output [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct + +(38) ColumnarToRow [codegen id : 12] +Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] + +(39) Filter [codegen id : 12] +Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] +Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) + +(40) Exchange +Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#34] + +(41) Sort [codegen id : 13] +Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] +Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 + +(42) SortMergeJoin +Left keys [2]: [cast(ss_ticket_number#26 as bigint), cast(ss_item_sk#25 as bigint)] +Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] +Join condition: None + +(43) Project [codegen id : 14] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (ss_quantity#27 - coalesce(sr_return_quantity#32, 0)) AS sales_cnt#35, CheckOverflow((promote_precision(cast(ss_ext_sales_price#28 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#33, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#36] +Input [13]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] + +(44) Union + +(45) HashAggregate [codegen id : 15] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] + +(46) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23, 5), true, [id=#37] + +(47) HashAggregate [codegen id : 16] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] + +(48) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] +ReadSchema: struct + +(49) ColumnarToRow [codegen id : 19] +Input [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] + +(50) Filter [codegen id : 19] +Input [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] +Condition : (isnotnull(ws_item_sk#39) AND isnotnull(ws_sold_date_sk#38)) + +(51) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(52) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(53) Project [codegen id : 19] +Output [9]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(54) ReusedExchange [Reuses operator id: 14] +Output [2]: [d_date_sk#13, d_year#14] + +(55) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(56) Project [codegen id : 19] +Output [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(57) Exchange +Input [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: hashpartitioning(cast(ws_order_number#40 as bigint), cast(ws_item_sk#39 as bigint), 5), true, [id=#43] + +(58) Sort [codegen id : 20] +Input [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Arguments: [cast(ws_order_number#40 as bigint) ASC NULLS FIRST, cast(ws_item_sk#39 as bigint) ASC NULLS FIRST], false, 0 + +(59) Scan parquet default.web_returns +Output [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] +ReadSchema: struct + +(60) ColumnarToRow [codegen id : 21] +Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] + +(61) Filter [codegen id : 21] +Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] +Condition : (isnotnull(wr_order_number#45) AND isnotnull(wr_item_sk#44)) + +(62) Exchange +Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] +Arguments: hashpartitioning(wr_order_number#45, wr_item_sk#44, 5), true, [id=#48] + +(63) Sort [codegen id : 22] +Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] +Arguments: [wr_order_number#45 ASC NULLS FIRST, wr_item_sk#44 ASC NULLS FIRST], false, 0 + +(64) SortMergeJoin +Left keys [2]: [cast(ws_order_number#40 as bigint), cast(ws_item_sk#39 as bigint)] +Right keys [2]: [wr_order_number#45, wr_item_sk#44] +Join condition: None + +(65) Project [codegen id : 23] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (ws_quantity#41 - coalesce(wr_return_quantity#46, 0)) AS sales_cnt#49, CheckOverflow((promote_precision(cast(ws_ext_sales_price#42 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#47, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#50] +Input [13]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] + +(66) Union + +(67) HashAggregate [codegen id : 24] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] + +(68) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23, 5), true, [id=#51] + +(69) HashAggregate [codegen id : 25] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] + +(70) HashAggregate [codegen id : 25] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#22, sales_amt#23] +Keys [5]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Functions [2]: [partial_sum(cast(sales_cnt#22 as bigint)), partial_sum(UnscaledValue(sales_amt#23))] +Aggregate Attributes [2]: [sum#52, sum#53] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#54, sum#55] + +(71) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#54, sum#55] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, 5), true, [id=#56] + +(72) HashAggregate [codegen id : 26] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#54, sum#55] +Keys [5]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Functions [2]: [sum(cast(sales_cnt#22 as bigint)), sum(UnscaledValue(sales_amt#23))] +Aggregate Attributes [2]: [sum(cast(sales_cnt#22 as bigint))#57, sum(UnscaledValue(sales_amt#23))#58] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum(cast(sales_cnt#22 as bigint))#57 AS sales_cnt#59, MakeDecimal(sum(UnscaledValue(sales_amt#23))#58,18,2) AS sales_amt#60] + +(73) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#59, sales_amt#60] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, 5), true, [id=#61] + +(74) Sort [codegen id : 27] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#59, sales_amt#60] +Arguments: [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_manufact_id#11 ASC NULLS FIRST], false, 0 + +(75) Scan parquet default.catalog_sales +Output [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] +ReadSchema: struct + +(76) ColumnarToRow [codegen id : 30] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] + +(77) Filter [codegen id : 30] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) + +(78) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(79) BroadcastHashJoin [codegen id : 30] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#62] +Join condition: None + +(80) Project [codegen id : 30] +Output [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Input [10]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(81) Scan parquet default.date_dim +Output [2]: [d_date_sk#67, d_year#68] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(82) ColumnarToRow [codegen id : 29] +Input [2]: [d_date_sk#67, d_year#68] + +(83) Filter [codegen id : 29] +Input [2]: [d_date_sk#67, d_year#68] +Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2001)) AND isnotnull(d_date_sk#67)) + +(84) BroadcastExchange +Input [2]: [d_date_sk#67, d_year#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] + +(85) BroadcastHashJoin [codegen id : 30] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#67] +Join condition: None + +(86) Project [codegen id : 30] +Output [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_date_sk#67, d_year#68] + +(87) Exchange +Input [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), true, [id=#70] + +(88) Sort [codegen id : 31] +Input [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST], false, 0 + +(89) ReusedExchange [Reuses operator id: 22] +Output [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] + +(90) Sort [codegen id : 33] +Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] +Arguments: [cr_order_number#18 ASC NULLS FIRST, cr_item_sk#17 ASC NULLS FIRST], false, 0 + +(91) SortMergeJoin +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#18, cr_item_sk#17] +Join condition: None + +(92) Project [codegen id : 34] +Output [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, (cs_quantity#4 - coalesce(cr_return_quantity#19, 0)) AS sales_cnt#22, CheckOverflow((promote_precision(cast(cs_ext_sales_price#5 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#20, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#23] +Input [13]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68, cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] + +(93) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] +ReadSchema: struct + +(94) ColumnarToRow [codegen id : 37] +Input [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] + +(95) Filter [codegen id : 37] +Input [5]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28] +Condition : (isnotnull(ss_item_sk#25) AND isnotnull(ss_sold_date_sk#24)) + +(96) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(97) BroadcastHashJoin [codegen id : 37] +Left keys [1]: [ss_item_sk#25] +Right keys [1]: [i_item_sk#62] +Join condition: None + +(98) Project [codegen id : 37] +Output [9]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Input [10]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(99) ReusedExchange [Reuses operator id: 84] +Output [2]: [d_date_sk#67, d_year#68] + +(100) BroadcastHashJoin [codegen id : 37] +Left keys [1]: [ss_sold_date_sk#24] +Right keys [1]: [d_date_sk#67] +Join condition: None + +(101) Project [codegen id : 37] +Output [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Input [11]: [ss_sold_date_sk#24, ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_date_sk#67, d_year#68] + +(102) Exchange +Input [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: hashpartitioning(cast(ss_ticket_number#26 as bigint), cast(ss_item_sk#25 as bigint), 5), true, [id=#71] + +(103) Sort [codegen id : 38] +Input [9]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: [cast(ss_ticket_number#26 as bigint) ASC NULLS FIRST, cast(ss_item_sk#25 as bigint) ASC NULLS FIRST], false, 0 + +(104) ReusedExchange [Reuses operator id: 40] +Output [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] + +(105) Sort [codegen id : 40] +Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] +Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 + +(106) SortMergeJoin +Left keys [2]: [cast(ss_ticket_number#26 as bigint), cast(ss_item_sk#25 as bigint)] +Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] +Join condition: None + +(107) Project [codegen id : 41] +Output [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, (ss_quantity#27 - coalesce(sr_return_quantity#32, 0)) AS sales_cnt#72, CheckOverflow((promote_precision(cast(ss_ext_sales_price#28 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#33, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#73] +Input [13]: [ss_item_sk#25, ss_ticket_number#26, ss_quantity#27, ss_ext_sales_price#28, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68, sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] + +(108) Union + +(109) HashAggregate [codegen id : 42] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] + +(110) Exchange +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Arguments: hashpartitioning(d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23, 5), true, [id=#74] + +(111) HashAggregate [codegen id : 43] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] + +(112) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] +ReadSchema: struct + +(113) ColumnarToRow [codegen id : 46] +Input [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] + +(114) Filter [codegen id : 46] +Input [5]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42] +Condition : (isnotnull(ws_item_sk#39) AND isnotnull(ws_sold_date_sk#38)) + +(115) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(116) BroadcastHashJoin [codegen id : 46] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#62] +Join condition: None + +(117) Project [codegen id : 46] +Output [9]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Input [10]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] + +(118) ReusedExchange [Reuses operator id: 84] +Output [2]: [d_date_sk#67, d_year#68] + +(119) BroadcastHashJoin [codegen id : 46] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#67] +Join condition: None + +(120) Project [codegen id : 46] +Output [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Input [11]: [ws_sold_date_sk#38, ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_date_sk#67, d_year#68] + +(121) Exchange +Input [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: hashpartitioning(cast(ws_order_number#40 as bigint), cast(ws_item_sk#39 as bigint), 5), true, [id=#75] + +(122) Sort [codegen id : 47] +Input [9]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68] +Arguments: [cast(ws_order_number#40 as bigint) ASC NULLS FIRST, cast(ws_item_sk#39 as bigint) ASC NULLS FIRST], false, 0 + +(123) ReusedExchange [Reuses operator id: 62] +Output [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] + +(124) Sort [codegen id : 49] +Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] +Arguments: [wr_order_number#45 ASC NULLS FIRST, wr_item_sk#44 ASC NULLS FIRST], false, 0 + +(125) SortMergeJoin +Left keys [2]: [cast(ws_order_number#40 as bigint), cast(ws_item_sk#39 as bigint)] +Right keys [2]: [wr_order_number#45, wr_item_sk#44] +Join condition: None + +(126) Project [codegen id : 50] +Output [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, (ws_quantity#41 - coalesce(wr_return_quantity#46, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ws_ext_sales_price#42 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#47, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] +Input [13]: [ws_item_sk#39, ws_order_number#40, ws_quantity#41, ws_ext_sales_price#42, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, d_year#68, wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] + +(127) Union + +(128) HashAggregate [codegen id : 51] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] + +(129) Exchange +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Arguments: hashpartitioning(d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23, 5), true, [id=#78] + +(130) HashAggregate [codegen id : 52] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Keys [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] + +(131) HashAggregate [codegen id : 52] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#22, sales_amt#23] +Keys [5]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Functions [2]: [partial_sum(cast(sales_cnt#22 as bigint)), partial_sum(UnscaledValue(sales_amt#23))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sum#81, sum#82] + +(132) Exchange +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sum#81, sum#82] +Arguments: hashpartitioning(d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, 5), true, [id=#83] + +(133) HashAggregate [codegen id : 53] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sum#81, sum#82] +Keys [5]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Functions [2]: [sum(cast(sales_cnt#22 as bigint)), sum(UnscaledValue(sales_amt#23))] +Aggregate Attributes [2]: [sum(cast(sales_cnt#22 as bigint))#84, sum(UnscaledValue(sales_amt#23))#85] +Results [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sum(cast(sales_cnt#22 as bigint))#84 AS sales_cnt#86, MakeDecimal(sum(UnscaledValue(sales_amt#23))#85,18,2) AS sales_amt#87] + +(134) Exchange +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#86, sales_amt#87] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, 5), true, [id=#88] + +(135) Sort [codegen id : 54] +Input [7]: [d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#86, sales_amt#87] +Arguments: [i_brand_id#63 ASC NULLS FIRST, i_class_id#64 ASC NULLS FIRST, i_category_id#65 ASC NULLS FIRST, i_manufact_id#66 ASC NULLS FIRST], false, 0 + +(136) SortMergeJoin [codegen id : 55] +Left keys [4]: [i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Right keys [4]: [i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#59 as decimal(17,2))) / promote_precision(cast(sales_cnt#86 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) + +(137) Project [codegen id : 55] +Output [10]: [d_year#68 AS prev_year#89, d_year#14 AS year#90, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#86 AS prev_yr_cnt#91, sales_cnt#59 AS curr_yr_cnt#92, (sales_cnt#59 - sales_cnt#86) AS sales_cnt_diff#93, CheckOverflow((promote_precision(cast(sales_amt#60 as decimal(19,2))) - promote_precision(cast(sales_amt#87 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#94] +Input [14]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#59, sales_amt#60, d_year#68, i_brand_id#63, i_class_id#64, i_category_id#65, i_manufact_id#66, sales_cnt#86, sales_amt#87] + +(138) TakeOrderedAndProject +Input [10]: [prev_year#89, year#90, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, prev_yr_cnt#91, curr_yr_cnt#92, sales_cnt_diff#93, sales_amt_diff#94] +Arguments: 100, [sales_cnt_diff#93 ASC NULLS FIRST, sales_amt_diff#94 ASC NULLS FIRST], [prev_year#89, year#90, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, prev_yr_cnt#91, curr_yr_cnt#92, sales_cnt_diff#93, sales_amt_diff#94] + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/SPARK-33152_simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/SPARK-33152_simplified.txt new file mode 100644 index 000000000000..c482032577da --- /dev/null +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/SPARK-33152_simplified.txt @@ -0,0 +1,237 @@ +TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt] + WholeStageCodegen (55) + Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] + InputAdapter + WholeStageCodegen (27) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + WholeStageCodegen (26) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + WholeStageCodegen (25) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + WholeStageCodegen (24) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (16) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #4 + WholeStageCodegen (15) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (7) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + InputAdapter + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + WholeStageCodegen (4) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #5 + WholeStageCodegen (3) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (6) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #8 + WholeStageCodegen (5) + Filter [cr_item_sk,cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + WholeStageCodegen (14) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + InputAdapter + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + WholeStageCodegen (11) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #9 + WholeStageCodegen (10) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + WholeStageCodegen (13) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #10 + WholeStageCodegen (12) + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + WholeStageCodegen (23) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + InputAdapter + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + WholeStageCodegen (20) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #11 + WholeStageCodegen (19) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + WholeStageCodegen (22) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #12 + WholeStageCodegen (21) + Filter [wr_order_number,wr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + InputAdapter + WholeStageCodegen (54) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + WholeStageCodegen (53) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + WholeStageCodegen (52) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + WholeStageCodegen (51) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (43) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #16 + WholeStageCodegen (42) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (34) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + InputAdapter + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + WholeStageCodegen (31) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #17 + WholeStageCodegen (30) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + BroadcastExchange #18 + WholeStageCodegen (29) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (33) + Sort [cr_order_number,cr_item_sk] + InputAdapter + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + WholeStageCodegen (41) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + InputAdapter + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + WholeStageCodegen (38) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #19 + WholeStageCodegen (37) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #18 + WholeStageCodegen (40) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + WholeStageCodegen (50) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + InputAdapter + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + WholeStageCodegen (47) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #20 + WholeStageCodegen (46) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #18 + WholeStageCodegen (49) + Sort [wr_order_number,wr_item_sk] + InputAdapter + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/SPARK-33152_explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/SPARK-33152_explain.txt new file mode 100644 index 000000000000..f9bb2e4f17c4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/SPARK-33152_explain.txt @@ -0,0 +1,647 @@ +== Physical Plan == +TakeOrderedAndProject (117) ++- * Project (116) + +- * BroadcastHashJoin Inner BuildRight (115) + :- * HashAggregate (63) + : +- Exchange (62) + : +- * HashAggregate (61) + : +- * HashAggregate (60) + : +- Exchange (59) + : +- * HashAggregate (58) + : +- Union (57) + : :- * HashAggregate (41) + : : +- Exchange (40) + : : +- * HashAggregate (39) + : : +- Union (38) + : : :- * Project (22) + : : : +- * BroadcastHashJoin LeftOuter BuildRight (21) + : : : :- * Project (16) + : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : :- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * Filter (3) + : : : : : : +- * ColumnarToRow (2) + : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * Project (7) + : : : : : +- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.item (4) + : : : : +- BroadcastExchange (14) + : : : : +- * Filter (13) + : : : : +- * ColumnarToRow (12) + : : : : +- Scan parquet default.date_dim (11) + : : : +- BroadcastExchange (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_returns (17) + : : +- * Project (37) + : : +- * BroadcastHashJoin LeftOuter BuildRight (36) + : : :- * Project (31) + : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : :- * Project (28) + : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : :- * Filter (25) + : : : : : +- * ColumnarToRow (24) + : : : : : +- Scan parquet default.store_sales (23) + : : : : +- ReusedExchange (26) + : : : +- ReusedExchange (29) + : : +- BroadcastExchange (35) + : : +- * Filter (34) + : : +- * ColumnarToRow (33) + : : +- Scan parquet default.store_returns (32) + : +- * Project (56) + : +- * BroadcastHashJoin LeftOuter BuildRight (55) + : :- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * Project (47) + : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : :- * Filter (44) + : : : : +- * ColumnarToRow (43) + : : : : +- Scan parquet default.web_sales (42) + : : : +- ReusedExchange (45) + : : +- ReusedExchange (48) + : +- BroadcastExchange (54) + : +- * Filter (53) + : +- * ColumnarToRow (52) + : +- Scan parquet default.web_returns (51) + +- BroadcastExchange (114) + +- * HashAggregate (113) + +- Exchange (112) + +- * HashAggregate (111) + +- * HashAggregate (110) + +- Exchange (109) + +- * HashAggregate (108) + +- Union (107) + :- * HashAggregate (94) + : +- Exchange (93) + : +- * HashAggregate (92) + : +- Union (91) + : :- * Project (78) + : : +- * BroadcastHashJoin LeftOuter BuildRight (77) + : : :- * Project (75) + : : : +- * BroadcastHashJoin Inner BuildRight (74) + : : : :- * Project (69) + : : : : +- * BroadcastHashJoin Inner BuildRight (68) + : : : : :- * Filter (66) + : : : : : +- * ColumnarToRow (65) + : : : : : +- Scan parquet default.catalog_sales (64) + : : : : +- ReusedExchange (67) + : : : +- BroadcastExchange (73) + : : : +- * Filter (72) + : : : +- * ColumnarToRow (71) + : : : +- Scan parquet default.date_dim (70) + : : +- ReusedExchange (76) + : +- * Project (90) + : +- * BroadcastHashJoin LeftOuter BuildRight (89) + : :- * Project (87) + : : +- * BroadcastHashJoin Inner BuildRight (86) + : : :- * Project (84) + : : : +- * BroadcastHashJoin Inner BuildRight (83) + : : : :- * Filter (81) + : : : : +- * ColumnarToRow (80) + : : : : +- Scan parquet default.store_sales (79) + : : : +- ReusedExchange (82) + : : +- ReusedExchange (85) + : +- ReusedExchange (88) + +- * Project (106) + +- * BroadcastHashJoin LeftOuter BuildRight (105) + :- * Project (103) + : +- * BroadcastHashJoin Inner BuildRight (102) + : :- * Project (100) + : : +- * BroadcastHashJoin Inner BuildRight (99) + : : :- * Filter (97) + : : : +- * ColumnarToRow (96) + : : : +- Scan parquet default.web_sales (95) + : : +- ReusedExchange (98) + : +- ReusedExchange (101) + +- ReusedExchange (104) + + +(1) Scan parquet default.catalog_sales +Output [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 4] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] + +(3) Filter [codegen id : 4] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) + +(4) Scan parquet default.item +Output [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] + +(6) Filter [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] +Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_manufact_id#11)) + +(7) Project [codegen id : 1] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] + +(8) BroadcastExchange +Input [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] + +(9) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(10) Project [codegen id : 4] +Output [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(11) Scan parquet default.date_dim +Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#13, d_year#14] + +(13) Filter [codegen id : 2] +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) + +(14) BroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(15) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(16) Project [codegen id : 4] +Output [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(17) Scan parquet default.catalog_returns +Output [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +ReadSchema: struct + +(18) ColumnarToRow [codegen id : 3] +Input [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] + +(19) Filter [codegen id : 3] +Input [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] +Condition : (isnotnull(cr_item_sk#16) AND isnotnull(cr_order_number#17)) + +(20) BroadcastExchange +Input [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [id=#20] + +(21) BroadcastHashJoin [codegen id : 4] +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#17, cr_item_sk#16] +Join condition: None + +(22) Project [codegen id : 4] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (cs_quantity#4 - coalesce(cr_return_quantity#18, 0)) AS sales_cnt#21, CheckOverflow((promote_precision(cast(cs_ext_sales_price#5 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#19, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#22] +Input [13]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] + +(23) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] +ReadSchema: struct + +(24) ColumnarToRow [codegen id : 8] +Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] + +(25) Filter [codegen id : 8] +Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] +Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_date_sk#23)) + +(26) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(27) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_item_sk#24] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(28) Project [codegen id : 8] +Output [9]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(29) ReusedExchange [Reuses operator id: 14] +Output [2]: [d_date_sk#13, d_year#14] + +(30) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(31) Project [codegen id : 8] +Output [9]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(32) Scan parquet default.store_returns +Output [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct + +(33) ColumnarToRow [codegen id : 7] +Input [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] + +(34) Filter [codegen id : 7] +Input [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] +Condition : (isnotnull(sr_ticket_number#29) AND isnotnull(sr_item_sk#28)) + +(35) BroadcastExchange +Input [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#32] + +(36) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [cast(ss_ticket_number#25 as bigint), cast(ss_item_sk#24 as bigint)] +Right keys [2]: [sr_ticket_number#29, sr_item_sk#28] +Join condition: None + +(37) Project [codegen id : 8] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (ss_quantity#26 - coalesce(sr_return_quantity#30, 0)) AS sales_cnt#33, CheckOverflow((promote_precision(cast(ss_ext_sales_price#27 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#31, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#34] +Input [13]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] + +(38) Union + +(39) HashAggregate [codegen id : 9] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] + +(40) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22, 5), true, [id=#35] + +(41) HashAggregate [codegen id : 10] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] + +(42) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 14] +Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] + +(44) Filter [codegen id : 14] +Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] +Condition : (isnotnull(ws_item_sk#37) AND isnotnull(ws_sold_date_sk#36)) + +(45) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(46) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#6] +Join condition: None + +(47) Project [codegen id : 14] +Output [9]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Input [10]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] + +(48) ReusedExchange [Reuses operator id: 14] +Output [2]: [d_date_sk#13, d_year#14] + +(49) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#13] +Join condition: None + +(50) Project [codegen id : 14] +Output [9]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] +Input [11]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] + +(51) Scan parquet default.web_returns +Output [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] +ReadSchema: struct + +(52) ColumnarToRow [codegen id : 13] +Input [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] + +(53) Filter [codegen id : 13] +Input [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] +Condition : (isnotnull(wr_order_number#42) AND isnotnull(wr_item_sk#41)) + +(54) BroadcastExchange +Input [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#45] + +(55) BroadcastHashJoin [codegen id : 14] +Left keys [2]: [cast(ws_order_number#38 as bigint), cast(ws_item_sk#37 as bigint)] +Right keys [2]: [wr_order_number#42, wr_item_sk#41] +Join condition: None + +(56) Project [codegen id : 14] +Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (ws_quantity#39 - coalesce(wr_return_quantity#43, 0)) AS sales_cnt#46, CheckOverflow((promote_precision(cast(ws_ext_sales_price#40 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#44, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#47] +Input [13]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] + +(57) Union + +(58) HashAggregate [codegen id : 15] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] + +(59) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22, 5), true, [id=#48] + +(60) HashAggregate [codegen id : 16] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] + +(61) HashAggregate [codegen id : 16] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] +Keys [5]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Functions [2]: [partial_sum(cast(sales_cnt#21 as bigint)), partial_sum(UnscaledValue(sales_amt#22))] +Aggregate Attributes [2]: [sum#49, sum#50] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#51, sum#52] + +(62) Exchange +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#51, sum#52] +Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, 5), true, [id=#53] + +(63) HashAggregate [codegen id : 34] +Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#51, sum#52] +Keys [5]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Functions [2]: [sum(cast(sales_cnt#21 as bigint)), sum(UnscaledValue(sales_amt#22))] +Aggregate Attributes [2]: [sum(cast(sales_cnt#21 as bigint))#54, sum(UnscaledValue(sales_amt#22))#55] +Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum(cast(sales_cnt#21 as bigint))#54 AS sales_cnt#56, MakeDecimal(sum(UnscaledValue(sales_amt#22))#55,18,2) AS sales_amt#57] + +(64) Scan parquet default.catalog_sales +Output [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] +ReadSchema: struct + +(65) ColumnarToRow [codegen id : 20] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] + +(66) Filter [codegen id : 20] +Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) + +(67) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(68) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#58] +Join condition: None + +(69) Project [codegen id : 20] +Output [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Input [10]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(70) Scan parquet default.date_dim +Output [2]: [d_date_sk#63, d_year#64] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(71) ColumnarToRow [codegen id : 18] +Input [2]: [d_date_sk#63, d_year#64] + +(72) Filter [codegen id : 18] +Input [2]: [d_date_sk#63, d_year#64] +Condition : ((isnotnull(d_year#64) AND (d_year#64 = 2001)) AND isnotnull(d_date_sk#63)) + +(73) BroadcastExchange +Input [2]: [d_date_sk#63, d_year#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] + +(74) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#63] +Join condition: None + +(75) Project [codegen id : 20] +Output [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_date_sk#63, d_year#64] + +(76) ReusedExchange [Reuses operator id: 20] +Output [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] + +(77) BroadcastHashJoin [codegen id : 20] +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#17, cr_item_sk#16] +Join condition: None + +(78) Project [codegen id : 20] +Output [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, (cs_quantity#4 - coalesce(cr_return_quantity#18, 0)) AS sales_cnt#21, CheckOverflow((promote_precision(cast(cs_ext_sales_price#5 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#19, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#22] +Input [13]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64, cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] + +(79) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] +ReadSchema: struct + +(80) ColumnarToRow [codegen id : 24] +Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] + +(81) Filter [codegen id : 24] +Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] +Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_date_sk#23)) + +(82) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(83) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_item_sk#24] +Right keys [1]: [i_item_sk#58] +Join condition: None + +(84) Project [codegen id : 24] +Output [9]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Input [10]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(85) ReusedExchange [Reuses operator id: 73] +Output [2]: [d_date_sk#63, d_year#64] + +(86) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#63] +Join condition: None + +(87) Project [codegen id : 24] +Output [9]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64] +Input [11]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_date_sk#63, d_year#64] + +(88) ReusedExchange [Reuses operator id: 35] +Output [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] + +(89) BroadcastHashJoin [codegen id : 24] +Left keys [2]: [cast(ss_ticket_number#25 as bigint), cast(ss_item_sk#24 as bigint)] +Right keys [2]: [sr_ticket_number#29, sr_item_sk#28] +Join condition: None + +(90) Project [codegen id : 24] +Output [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, (ss_quantity#26 - coalesce(sr_return_quantity#30, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ss_ext_sales_price#27 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#31, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64, sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] + +(91) Union + +(92) HashAggregate [codegen id : 25] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] + +(93) Exchange +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Arguments: hashpartitioning(d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22, 5), true, [id=#68] + +(94) HashAggregate [codegen id : 26] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] + +(95) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] +ReadSchema: struct + +(96) ColumnarToRow [codegen id : 30] +Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] + +(97) Filter [codegen id : 30] +Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] +Condition : (isnotnull(ws_item_sk#37) AND isnotnull(ws_sold_date_sk#36)) + +(98) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(99) BroadcastHashJoin [codegen id : 30] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#58] +Join condition: None + +(100) Project [codegen id : 30] +Output [9]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Input [10]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] + +(101) ReusedExchange [Reuses operator id: 73] +Output [2]: [d_date_sk#63, d_year#64] + +(102) BroadcastHashJoin [codegen id : 30] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#63] +Join condition: None + +(103) Project [codegen id : 30] +Output [9]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64] +Input [11]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_date_sk#63, d_year#64] + +(104) ReusedExchange [Reuses operator id: 54] +Output [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] + +(105) BroadcastHashJoin [codegen id : 30] +Left keys [2]: [cast(ws_order_number#38 as bigint), cast(ws_item_sk#37 as bigint)] +Right keys [2]: [wr_order_number#42, wr_item_sk#41] +Join condition: None + +(106) Project [codegen id : 30] +Output [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, (ws_quantity#39 - coalesce(wr_return_quantity#43, 0)) AS sales_cnt#69, CheckOverflow((promote_precision(cast(ws_ext_sales_price#40 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#44, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#70] +Input [13]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64, wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] + +(107) Union + +(108) HashAggregate [codegen id : 31] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] + +(109) Exchange +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Arguments: hashpartitioning(d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22, 5), true, [id=#71] + +(110) HashAggregate [codegen id : 32] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] + +(111) HashAggregate [codegen id : 32] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] +Keys [5]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Functions [2]: [partial_sum(cast(sales_cnt#21 as bigint)), partial_sum(UnscaledValue(sales_amt#22))] +Aggregate Attributes [2]: [sum#72, sum#73] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum#74, sum#75] + +(112) Exchange +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum#74, sum#75] +Arguments: hashpartitioning(d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, 5), true, [id=#76] + +(113) HashAggregate [codegen id : 33] +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum#74, sum#75] +Keys [5]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Functions [2]: [sum(cast(sales_cnt#21 as bigint)), sum(UnscaledValue(sales_amt#22))] +Aggregate Attributes [2]: [sum(cast(sales_cnt#21 as bigint))#77, sum(UnscaledValue(sales_amt#22))#78] +Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum(cast(sales_cnt#21 as bigint))#77 AS sales_cnt#79, MakeDecimal(sum(UnscaledValue(sales_amt#22))#78,18,2) AS sales_amt#80] + +(114) BroadcastExchange +Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#79, sales_amt#80] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true], input[4, int, true]),false), [id=#81] + +(115) BroadcastHashJoin [codegen id : 34] +Left keys [4]: [i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] +Right keys [4]: [i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#56 as decimal(17,2))) / promote_precision(cast(sales_cnt#79 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) + +(116) Project [codegen id : 34] +Output [10]: [d_year#64 AS prev_year#82, d_year#14 AS year#83, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#79 AS prev_yr_cnt#84, sales_cnt#56 AS curr_yr_cnt#85, (sales_cnt#56 - sales_cnt#79) AS sales_cnt_diff#86, CheckOverflow((promote_precision(cast(sales_amt#57 as decimal(19,2))) - promote_precision(cast(sales_amt#80 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#87] +Input [14]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#56, sales_amt#57, d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#79, sales_amt#80] + +(117) TakeOrderedAndProject +Input [10]: [prev_year#82, year#83, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, prev_yr_cnt#84, curr_yr_cnt#85, sales_cnt_diff#86, sales_amt_diff#87] +Arguments: 100, [sales_cnt_diff#86 ASC NULLS FIRST, sales_amt_diff#87 ASC NULLS FIRST], [prev_year#82, year#83, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, prev_yr_cnt#84, curr_yr_cnt#85, sales_cnt_diff#86, sales_amt_diff#87] + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/SPARK-33152_simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/SPARK-33152_simplified.txt new file mode 100644 index 000000000000..9e004b842064 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/SPARK-33152_simplified.txt @@ -0,0 +1,180 @@ +TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt] + WholeStageCodegen (34) + Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + WholeStageCodegen (16) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #2 + WholeStageCodegen (15) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (10) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + WholeStageCodegen (9) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (4) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Filter [cr_item_sk,cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + WholeStageCodegen (8) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + WholeStageCodegen (14) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (13) + Filter [wr_order_number,wr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (33) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #10 + WholeStageCodegen (32) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #11 + WholeStageCodegen (31) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (26) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #12 + WholeStageCodegen (25) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (20) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (18) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #6 + WholeStageCodegen (24) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year] #13 + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #7 + WholeStageCodegen (30) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year] #13 + InputAdapter + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #8 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index 76204c504c0e..0d6fc0e21c3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -102,7 +102,16 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite { } private def isApproved(dir: File, actualSimplifiedPlan: String): Boolean = { - val file = new File(dir, "simplified.txt") + val baseFileName = "simplified.txt" + val prefix = if (SQLConf.get.useOptimizedConstraintPropagation) { + "SPARK-33152_" + } else "" + val goldenFileName = if (SQLConf.get.useOptimizedConstraintPropagation) { + if (new File(dir, prefix + baseFileName).exists()) { + prefix + baseFileName + } else baseFileName + } else baseFileName + val file = new File(dir, goldenFileName) val expected = FileUtils.readFileToString(file, StandardCharsets.UTF_8) expected == actualSimplifiedPlan } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index 13b22dba1168..d438b7fb15c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -56,7 +56,7 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils { } } - override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + override def withSQLConf[T](pairs: (String, String)*)(f: => T): T = { pairs.foreach { case (k, v) => SQLConf.get.setConfString(k, v) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 7be15e9d8700..50b13d064674 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -241,7 +241,7 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } - protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + protected override def withSQLConf[T](pairs: (String, String)*)(f: => T): T = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index d7129bcb37e6..3eec0f562d94 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -82,7 +82,7 @@ class HiveSerDeSuite extends HiveComparisonTest with PlanTest with BeforeAndAfte } // Make sure we set the config values to TestHive.conf. - override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = + override def withSQLConf[T](pairs: (String, String)*)(f: => T): T = SQLConf.withExistingConf(TestHive.conf)(super.withSQLConf(pairs: _*)(f)) test("Test the default fileformat for Hive-serde tables") {