From c1e995ac95cb905670c9ec1dc343d4bfc953a19e Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Thu, 13 May 2021 12:52:26 +0000 Subject: [PATCH] [SPARK-35350][SQL] Add code-gen for left semi sort merge join ### What changes were proposed in this pull request? As title. This PR is to add code-gen support for LEFT SEMI sort merge join. The main change is to add `semiJoin` code path in `SortMergeJoinExec.doProduce()` and introduce `onlyBufferFirstMatchedRow` in `SortMergeJoinExec.genScanner()`. The latter is for left semi sort merge join without condition. For this kind of query, we don't need to buffer all matched rows, but only the first one (this is same as non-code-gen code path). Example query: ``` val df1 = spark.range(10).select($"id".as("k1")) val df2 = spark.range(4).select($"id".as("k2")) val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_semi") ``` Example of generated code for the query: ``` == Subtree 5 / 5 (maxMethodCodeSize:302; maxConstantPoolSize:156(0.24% used); numInnerClasses:0) == *(5) Project [id#0L AS k1#2L] +- *(5) SortMergeJoin [id#0L], [k2#6L], LeftSemi :- *(2) Sort [id#0L ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(id#0L, 5), ENSURE_REQUIREMENTS, [id=#27] : +- *(1) Range (0, 10, step=1, splits=2) +- *(4) Sort [k2#6L ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(k2#6L, 5), ENSURE_REQUIREMENTS, [id=#33] +- *(3) Project [id#4L AS k2#6L] +- *(3) Range (0, 4, step=1, splits=2) Generated code: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIteratorForCodegenStage5(references); /* 003 */ } /* 004 */ /* 005 */ // codegenStageId=5 /* 006 */ final class GeneratedIteratorForCodegenStage5 extends org.apache.spark.sql.execution.BufferedRowIterator { /* 007 */ private Object[] references; /* 008 */ private scala.collection.Iterator[] inputs; /* 009 */ private scala.collection.Iterator smj_streamedInput_0; /* 010 */ private scala.collection.Iterator smj_bufferedInput_0; /* 011 */ private InternalRow smj_streamedRow_0; /* 012 */ private InternalRow smj_bufferedRow_0; /* 013 */ private long smj_value_2; /* 014 */ private org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray smj_matches_0; /* 015 */ private long smj_value_3; /* 016 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] smj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2]; /* 017 */ /* 018 */ public GeneratedIteratorForCodegenStage5(Object[] references) { /* 019 */ this.references = references; /* 020 */ } /* 021 */ /* 022 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 023 */ partitionIndex = index; /* 024 */ this.inputs = inputs; /* 025 */ smj_streamedInput_0 = inputs[0]; /* 026 */ smj_bufferedInput_0 = inputs[1]; /* 027 */ /* 028 */ smj_matches_0 = new org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray(1, 2147483647); /* 029 */ smj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0); /* 030 */ smj_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0); /* 031 */ /* 032 */ } /* 033 */ /* 034 */ private boolean findNextJoinRows( /* 035 */ scala.collection.Iterator streamedIter, /* 036 */ scala.collection.Iterator bufferedIter) { /* 037 */ smj_streamedRow_0 = null; /* 038 */ int comp = 0; /* 039 */ while (smj_streamedRow_0 == null) { /* 040 */ if (!streamedIter.hasNext()) return false; /* 041 */ smj_streamedRow_0 = (InternalRow) streamedIter.next(); /* 042 */ long smj_value_0 = smj_streamedRow_0.getLong(0); /* 043 */ if (false) { /* 044 */ smj_streamedRow_0 = null; /* 045 */ continue; /* 046 */ /* 047 */ } /* 048 */ if (!smj_matches_0.isEmpty()) { /* 049 */ comp = 0; /* 050 */ if (comp == 0) { /* 051 */ comp = (smj_value_0 > smj_value_3 ? 1 : smj_value_0 < smj_value_3 ? -1 : 0); /* 052 */ } /* 053 */ /* 054 */ if (comp == 0) { /* 055 */ return true; /* 056 */ } /* 057 */ smj_matches_0.clear(); /* 058 */ } /* 059 */ /* 060 */ do { /* 061 */ if (smj_bufferedRow_0 == null) { /* 062 */ if (!bufferedIter.hasNext()) { /* 063 */ smj_value_3 = smj_value_0; /* 064 */ return !smj_matches_0.isEmpty(); /* 065 */ } /* 066 */ smj_bufferedRow_0 = (InternalRow) bufferedIter.next(); /* 067 */ long smj_value_1 = smj_bufferedRow_0.getLong(0); /* 068 */ if (false) { /* 069 */ smj_bufferedRow_0 = null; /* 070 */ continue; /* 071 */ } /* 072 */ smj_value_2 = smj_value_1; /* 073 */ } /* 074 */ /* 075 */ comp = 0; /* 076 */ if (comp == 0) { /* 077 */ comp = (smj_value_0 > smj_value_2 ? 1 : smj_value_0 < smj_value_2 ? -1 : 0); /* 078 */ } /* 079 */ /* 080 */ if (comp > 0) { /* 081 */ smj_bufferedRow_0 = null; /* 082 */ } else if (comp < 0) { /* 083 */ if (!smj_matches_0.isEmpty()) { /* 084 */ smj_value_3 = smj_value_0; /* 085 */ return true; /* 086 */ } else { /* 087 */ smj_streamedRow_0 = null; /* 088 */ } /* 089 */ } else { /* 090 */ if (smj_matches_0.isEmpty()) { /* 091 */ smj_matches_0.add((UnsafeRow) smj_bufferedRow_0); /* 092 */ } /* 093 */ /* 094 */ smj_bufferedRow_0 = null; /* 095 */ } /* 096 */ } while (smj_streamedRow_0 != null); /* 097 */ } /* 098 */ return false; // unreachable /* 099 */ } /* 100 */ /* 101 */ protected void processNext() throws java.io.IOException { /* 102 */ while (findNextJoinRows(smj_streamedInput_0, smj_bufferedInput_0)) { /* 103 */ long smj_value_4 = -1L; /* 104 */ smj_value_4 = smj_streamedRow_0.getLong(0); /* 105 */ scala.collection.Iterator smj_iterator_0 = smj_matches_0.generateIterator(); /* 106 */ boolean smj_hasOutputRow_0 = false; /* 107 */ /* 108 */ while (!smj_hasOutputRow_0 && smj_iterator_0.hasNext()) { /* 109 */ InternalRow smj_bufferedRow_1 = (InternalRow) smj_iterator_0.next(); /* 110 */ /* 111 */ smj_hasOutputRow_0 = true; /* 112 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1); /* 113 */ /* 114 */ // common sub-expressions /* 115 */ /* 116 */ smj_mutableStateArray_0[1].reset(); /* 117 */ /* 118 */ smj_mutableStateArray_0[1].write(0, smj_value_4); /* 119 */ append((smj_mutableStateArray_0[1].getRow()).copy()); /* 120 */ /* 121 */ } /* 122 */ if (shouldStop()) return; /* 123 */ } /* 124 */ ((org.apache.spark.sql.execution.joins.SortMergeJoinExec) references[1] /* plan */).cleanupResources(); /* 125 */ } /* 126 */ /* 127 */ } ``` ### Why are the changes needed? Improve query CPU performance. Test with one query: ``` def sortMergeJoin(): Unit = { val N = 2 << 20 codegenBenchmark("left semi sort merge join", N) { val df1 = spark.range(N).selectExpr(s"id * 2 as k1") val df2 = spark.range(N).selectExpr(s"id * 3 as k2") val df = df1.join(df2, col("k1") === col("k2"), "left_semi") assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined) df.noop() } } ``` Seeing 30% of run-time improvement: ``` Running benchmark: left semi sort merge join Running case: left semi sort merge join code-gen off Stopped after 2 iterations, 1369 ms Running case: left semi sort merge join code-gen on Stopped after 5 iterations, 2743 ms Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.16 Intel(R) Core(TM) i9-9980HK CPU 2.40GHz left semi sort merge join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ left semi sort merge join code-gen off 676 685 13 3.1 322.2 1.0X left semi sort merge join code-gen on 524 549 32 4.0 249.7 1.3X ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added unit test in `WholeStageCodegenSuite.scala` and `ExistenceJoinSuite.scala`. Closes #32528 from c21/smj-left-semi. Authored-by: Cheng Su Signed-off-by: Wenchen Fan --- .../execution/joins/SortMergeJoinExec.scala | 212 +++++++++----- .../q10.sf100/explain.txt | 38 +-- .../q10.sf100/simplified.txt | 100 +++---- .../approved-plans-v1_4/q10.sf100/explain.txt | 50 ++-- .../q10.sf100/simplified.txt | 85 +++--- .../q14a.sf100/explain.txt | 154 +++++------ .../q14a.sf100/simplified.txt | 164 +++++------ .../q14b.sf100/explain.txt | 128 ++++----- .../q14b.sf100/simplified.txt | 147 +++++----- .../approved-plans-v1_4/q16.sf100/explain.txt | 4 +- .../q16.sf100/simplified.txt | 5 +- .../approved-plans-v1_4/q16/explain.txt | 4 +- .../approved-plans-v1_4/q16/simplified.txt | 5 +- .../q23a.sf100/explain.txt | 16 +- .../q23a.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q23a/explain.txt | 8 +- .../approved-plans-v1_4/q23a/simplified.txt | 10 +- .../q23b.sf100/explain.txt | 110 ++++---- .../q23b.sf100/simplified.txt | 162 ++++++----- .../approved-plans-v1_4/q23b/explain.txt | 74 ++--- .../approved-plans-v1_4/q23b/simplified.txt | 92 ++++--- .../approved-plans-v1_4/q35.sf100/explain.txt | 52 ++-- .../q35.sf100/simplified.txt | 91 +++--- .../approved-plans-v1_4/q38.sf100/explain.txt | 8 +- .../q38.sf100/simplified.txt | 10 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 40 +-- .../q69.sf100/simplified.txt | 79 +++--- .../approved-plans-v1_4/q94.sf100/explain.txt | 4 +- .../q94.sf100/simplified.txt | 5 +- .../approved-plans-v1_4/q94/explain.txt | 4 +- .../approved-plans-v1_4/q94/simplified.txt | 5 +- .../approved-plans-v1_4/q95.sf100/explain.txt | 66 ++--- .../q95.sf100/simplified.txt | 88 +++--- .../approved-plans-v1_4/q95/explain.txt | 68 ++--- .../approved-plans-v1_4/q95/simplified.txt | 88 +++--- .../q10a.sf100/explain.txt | 42 +-- .../q10a.sf100/simplified.txt | 82 +++--- .../approved-plans-v2_7/q14.sf100/explain.txt | 128 ++++----- .../q14.sf100/simplified.txt | 147 +++++----- .../q14a.sf100/explain.txt | 260 +++++++++--------- .../q14a.sf100/simplified.txt | 216 ++++++++------- .../approved-plans-v2_7/q35.sf100/explain.txt | 52 ++-- .../q35.sf100/simplified.txt | 91 +++--- .../q35a.sf100/explain.txt | 52 ++-- .../q35a.sf100/simplified.txt | 94 ++++--- .../execution/WholeStageCodegenSuite.scala | 22 ++ .../execution/joins/ExistenceJoinSuite.scala | 2 +- 47 files changed, 1797 insertions(+), 1587 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 291ab29f1b3ba..db6626bd18abc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -105,8 +105,18 @@ case class SortMergeJoinExec( sqlContext.conf.sortMergeJoinExecBufferSpillThreshold } + // Flag to only buffer first matched row, to avoid buffering unnecessary rows. + private val onlyBufferFirstMatchedRow = (joinType, condition) match { + case (LeftExistence(_), None) => true + case _ => false + } + private def getInMemoryThreshold: Int = { - sqlContext.conf.sortMergeJoinExecBufferInMemoryThreshold + if (onlyBufferFirstMatchedRow) { + 1 + } else { + sqlContext.conf.sortMergeJoinExecBufferInMemoryThreshold + } } protected override def doExecute(): RDD[InternalRow] = { @@ -236,7 +246,7 @@ case class SortMergeJoinExec( inMemoryThreshold, spillThreshold, cleanupResources, - condition.isEmpty + onlyBufferFirstMatchedRow ) private[this] val joinRow = new JoinedRow @@ -273,7 +283,7 @@ case class SortMergeJoinExec( inMemoryThreshold, spillThreshold, cleanupResources, - condition.isEmpty + onlyBufferFirstMatchedRow ) private[this] val joinRow = new JoinedRow @@ -317,7 +327,7 @@ case class SortMergeJoinExec( inMemoryThreshold, spillThreshold, cleanupResources, - condition.isEmpty + onlyBufferFirstMatchedRow ) private[this] val joinRow = new JoinedRow @@ -354,7 +364,7 @@ case class SortMergeJoinExec( } private lazy val ((streamedPlan, streamedKeys), (bufferedPlan, bufferedKeys)) = joinType match { - case _: InnerLike | LeftOuter => ((left, leftKeys), (right, rightKeys)) + case _: InnerLike | LeftOuter | LeftSemi => ((left, leftKeys), (right, rightKeys)) case RightOuter => ((right, rightKeys), (left, leftKeys)) case x => throw new IllegalArgumentException( @@ -365,7 +375,7 @@ case class SortMergeJoinExec( private lazy val bufferedOutput = bufferedPlan.output override def supportCodegen: Boolean = joinType match { - case _: InnerLike | LeftOuter | RightOuter => true + case _: InnerLike | LeftOuter | RightOuter | LeftSemi => true case _ => false } @@ -435,7 +445,7 @@ case class SortMergeJoinExec( // Handle the case when streamed rows has any NULL keys. val handleStreamedAnyNull = joinType match { - case _: InnerLike => + case _: InnerLike | LeftSemi => // Skip streamed row. s""" |$streamedRow = null; @@ -457,7 +467,7 @@ case class SortMergeJoinExec( // Handle the case when streamed keys has no match with buffered side. val handleStreamedWithoutMatch = joinType match { - case _: InnerLike => + case _: InnerLike | LeftSemi => // Skip streamed row. s"$streamedRow = null;" case LeftOuter | RightOuter => @@ -468,6 +478,17 @@ case class SortMergeJoinExec( s"SortMergeJoin.genScanner should not take $x as the JoinType") } + val addRowToBuffer = + if (onlyBufferFirstMatchedRow) { + s""" + |if ($matches.isEmpty()) { + | $matches.add((UnsafeRow) $bufferedRow); + |} + """.stripMargin + } else { + s"$matches.add((UnsafeRow) $bufferedRow);" + } + // Generate a function to scan both streamed and buffered sides to find a match. // Return whether a match is found. // @@ -483,17 +504,18 @@ case class SortMergeJoinExec( // The function has the following step: // - Step 1: Find the next `streamedRow` with non-null join keys. // For `streamedRow` with null join keys (`handleStreamedAnyNull`): - // 1. Inner join: skip the row. `matches` will be cleared later when hitting the - // next `streamedRow` with non-null join keys. + // 1. Inner and Left Semi join: skip the row. `matches` will be cleared later when + // hitting the next `streamedRow` with non-null join + // keys. // 2. Left/Right Outer join: clear the previous `matches` if needed, keep the row, // and return false. // // - Step 2: Find the `matches` from buffered side having same join keys with `streamedRow`. // Clear `matches` if we hit a new `streamedRow`, as we need to find new matches. // Use `bufferedRow` to iterate buffered side to put all matched rows into - // `matches`. Return true when getting all matched rows. + // `matches` (`addRowToBuffer`). Return true when getting all matched rows. // For `streamedRow` without `matches` (`handleStreamedWithoutMatch`): - // 1. Inner join: skip the row. + // 1. Inner and Left Semi join: skip the row. // 2. Left/Right Outer join: keep the row and return false (with `matches` being // empty). ctx.addNewFunction("findNextJoinRows", @@ -543,7 +565,7 @@ case class SortMergeJoinExec( | $handleStreamedWithoutMatch | } | } else { - | $matches.add((UnsafeRow) $bufferedRow); + | $addRowToBuffer | $bufferedRow = null; | } | } while ($streamedRow != null); @@ -639,19 +661,22 @@ case class SortMergeJoinExec( streamedVars ++ bufferedVars case RightOuter => bufferedVars ++ streamedVars + case LeftSemi => + streamedVars case x => throw new IllegalArgumentException( s"SortMergeJoin.doProduce should not take $x as the JoinType") } - val (beforeLoop, condCheck) = if (condition.isDefined) { + val (streamedBeforeLoop, condCheck) = if (condition.isDefined) { // Split the code of creating variables based on whether it's used by condition or not. val loaded = ctx.freshName("loaded") val (streamedBefore, streamedAfter) = splitVarsByCondition(streamedOutput, streamedVars) val (bufferedBefore, bufferedAfter) = splitVarsByCondition(bufferedOutput, bufferedVars) // Generate code for condition - ctx.currentVars = resultVars - val cond = BindReferences.bindReference(condition.get, output).genCode(ctx) + ctx.currentVars = streamedVars ++ bufferedVars + val cond = BindReferences.bindReference( + condition.get, streamedPlan.output ++ bufferedPlan.output).genCode(ctx) // evaluate the columns those used by condition before loop val before = s""" @@ -674,65 +699,129 @@ case class SortMergeJoinExec( |} |$bufferedAfter """.stripMargin - (before, checking) + (before, checking.trim) } else { (evaluateVariables(streamedVars), "") } - val thisPlan = ctx.addReferenceObj("plan", this) - val eagerCleanup = s"$thisPlan.cleanupResources();" - - lazy val innerJoin = + val beforeLoop = s""" - |while (findNextJoinRows($streamedInput, $bufferedInput)) { - | ${streamedVarDecl.mkString("\n")} - | ${beforeLoop.trim} - | scala.collection.Iterator $iterator = $matches.generateIterator(); - | while ($iterator.hasNext()) { - | InternalRow $bufferedRow = (InternalRow) $iterator.next(); - | ${condCheck.trim} - | $numOutput.add(1); - | ${consume(ctx, resultVars)} - | } - | if (shouldStop()) return; - |} - |$eagerCleanup - """.stripMargin - - lazy val outerJoin = { - val hasOutputRow = ctx.freshName("hasOutputRow") + |${streamedVarDecl.mkString("\n")} + |${streamedBeforeLoop.trim} + |scala.collection.Iterator $iterator = $matches.generateIterator(); + """.stripMargin + val outputRow = s""" - |while ($streamedInput.hasNext()) { - | findNextJoinRows($streamedInput, $bufferedInput); - | ${streamedVarDecl.mkString("\n")} - | ${beforeLoop.trim} - | scala.collection.Iterator $iterator = $matches.generateIterator(); - | boolean $hasOutputRow = false; - | - | // the last iteration of this loop is to emit an empty row if there is no matched rows. - | while ($iterator.hasNext() || !$hasOutputRow) { - | InternalRow $bufferedRow = $iterator.hasNext() ? - | (InternalRow) $iterator.next() : null; - | ${condCheck.trim} - | $hasOutputRow = true; - | $numOutput.add(1); - | ${consume(ctx, resultVars)} - | } - | if (shouldStop()) return; - |} - |$eagerCleanup + |$numOutput.add(1); + |${consume(ctx, resultVars)} """.stripMargin - } + val findNextJoinRows = s"findNextJoinRows($streamedInput, $bufferedInput)" + val thisPlan = ctx.addReferenceObj("plan", this) + val eagerCleanup = s"$thisPlan.cleanupResources();" joinType match { - case _: InnerLike => innerJoin - case LeftOuter | RightOuter => outerJoin + case _: InnerLike => + codegenInner(findNextJoinRows, beforeLoop, iterator, bufferedRow, condCheck, outputRow, + eagerCleanup) + case LeftOuter | RightOuter => + codegenOuter(streamedInput, findNextJoinRows, beforeLoop, iterator, bufferedRow, condCheck, + ctx.freshName("hasOutputRow"), outputRow, eagerCleanup) + case LeftSemi => + codegenSemi(findNextJoinRows, beforeLoop, iterator, bufferedRow, condCheck, + ctx.freshName("hasOutputRow"), outputRow, eagerCleanup) case x => throw new IllegalArgumentException( s"SortMergeJoin.doProduce should not take $x as the JoinType") } } + /** + * Generates the code for Inner join. + */ + private def codegenInner( + findNextJoinRows: String, + beforeLoop: String, + matchIterator: String, + bufferedRow: String, + conditionCheck: String, + outputRow: String, + eagerCleanup: String): String = { + s""" + |while ($findNextJoinRows) { + | $beforeLoop + | while ($matchIterator.hasNext()) { + | InternalRow $bufferedRow = (InternalRow) $matchIterator.next(); + | $conditionCheck + | $outputRow + | } + | if (shouldStop()) return; + |} + |$eagerCleanup + """.stripMargin + } + + /** + * Generates the code for Left or Right Outer join. + */ + private def codegenOuter( + streamedInput: String, + findNextJoinRows: String, + beforeLoop: String, + matchIterator: String, + bufferedRow: String, + conditionCheck: String, + hasOutputRow: String, + outputRow: String, + eagerCleanup: String): String = { + s""" + |while ($streamedInput.hasNext()) { + | $findNextJoinRows; + | $beforeLoop + | boolean $hasOutputRow = false; + | + | // the last iteration of this loop is to emit an empty row if there is no matched rows. + | while ($matchIterator.hasNext() || !$hasOutputRow) { + | InternalRow $bufferedRow = $matchIterator.hasNext() ? + | (InternalRow) $matchIterator.next() : null; + | $conditionCheck + | $hasOutputRow = true; + | $outputRow + | } + | if (shouldStop()) return; + |} + |$eagerCleanup + """.stripMargin + } + + /** + * Generates the code for Left Semi join. + */ + private def codegenSemi( + findNextJoinRows: String, + beforeLoop: String, + matchIterator: String, + bufferedRow: String, + conditionCheck: String, + hasOutputRow: String, + outputRow: String, + eagerCleanup: String): String = { + s""" + |while ($findNextJoinRows) { + | $beforeLoop + | boolean $hasOutputRow = false; + | + | while (!$hasOutputRow && $matchIterator.hasNext()) { + | InternalRow $bufferedRow = (InternalRow) $matchIterator.next(); + | $conditionCheck + | $hasOutputRow = true; + | $outputRow + | } + | if (shouldStop()) return; + |} + |$eagerCleanup + """.stripMargin + } + override protected def withNewChildrenInternal( newLeft: SparkPlan, newRight: SparkPlan): SortMergeJoinExec = copy(left = newLeft, right = newRight) @@ -783,8 +872,7 @@ private[joins] class SortMergeJoinScanner( private[this] var matchJoinKey: InternalRow = _ /** Buffered rows from the buffered side of the join. This is empty if there are no matches. */ private[this] val bufferedMatches: ExternalAppendOnlyUnsafeRowArray = - new ExternalAppendOnlyUnsafeRowArray(if (onlyBufferFirstMatch) 1 else inMemoryThreshold, - spillThreshold) + new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold) // Initialization (note: do _not_ want to advance streamed here). advancedBufferedToRowWithNullFreeJoinKey() diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt index 1fdcba1ed4de4..214c9260016bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt @@ -9,8 +9,8 @@ TakeOrderedAndProject (52) : +- * Project (42) : +- * BroadcastHashJoin Inner BuildRight (41) : :- * Project (35) - : : +- SortMergeJoin LeftSemi (34) - : : :- SortMergeJoin LeftSemi (25) + : : +- * SortMergeJoin LeftSemi (34) + : : :- * SortMergeJoin LeftSemi (25) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) @@ -158,7 +158,7 @@ Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#17] Input [1]: [customer_sk#12] Arguments: [customer_sk#12 ASC NULLS FIRST], false, 0 -(25) SortMergeJoin +(25) SortMergeJoin [codegen id : 8] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#12] Join condition: None @@ -171,22 +171,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 9] +(27) ColumnarToRow [codegen id : 10] Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] -(28) Filter [codegen id : 9] +(28) Filter [codegen id : 10] Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] Condition : isnotnull(ss_customer_sk#18) (29) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#20] -(30) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#19] Right keys [1]: [d_date_sk#20] Join condition: None -(31) Project [codegen id : 9] +(31) Project [codegen id : 10] Output [1]: [ss_customer_sk#18 AS customer_sk#21] Input [3]: [ss_customer_sk#18, ss_sold_date_sk#19, d_date_sk#20] @@ -194,16 +194,16 @@ Input [3]: [ss_customer_sk#18, ss_sold_date_sk#19, d_date_sk#20] Input [1]: [customer_sk#21] Arguments: hashpartitioning(customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#22] -(33) Sort [codegen id : 10] +(33) Sort [codegen id : 11] Input [1]: [customer_sk#21] Arguments: [customer_sk#21 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(34) SortMergeJoin [codegen id : 13] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#21] Join condition: None -(35) Project [codegen id : 12] +(35) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -214,14 +214,14 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 11] +(37) ColumnarToRow [codegen id : 12] Input [2]: [ca_address_sk#23, ca_county#24] -(38) Filter [codegen id : 11] +(38) Filter [codegen id : 12] Input [2]: [ca_address_sk#23, ca_county#24] Condition : (ca_county#24 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#23)) -(39) Project [codegen id : 11] +(39) Project [codegen id : 12] Output [1]: [ca_address_sk#23] Input [2]: [ca_address_sk#23, ca_county#24] @@ -229,12 +229,12 @@ Input [2]: [ca_address_sk#23, ca_county#24] Input [1]: [ca_address_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] -(41) BroadcastHashJoin [codegen id : 12] +(41) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#23] Join condition: None -(42) Project [codegen id : 12] +(42) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23] @@ -256,16 +256,16 @@ Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_stat Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Condition : isnotnull(cd_demo_sk#27) -(47) BroadcastHashJoin [codegen id : 13] +(47) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#27] Join condition: None -(48) Project [codegen id : 13] +(48) Project [codegen id : 14] Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -(49) HashAggregate [codegen id : 13] +(49) HashAggregate [codegen id : 14] Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [partial_count(1)] @@ -276,7 +276,7 @@ Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_pur Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#38] -(51) HashAggregate [codegen id : 14] +(51) HashAggregate [codegen id : 15] Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/simplified.txt index 48e454cf87d2d..af1e7c10e8e4d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/simplified.txt @@ -1,67 +1,71 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (14) + WholeStageCodegen (15) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (13) + WholeStageCodegen (14) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (12) + WholeStageCodegen (13) Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - SortMergeJoin [c_customer_sk,customer_sk] - SortMergeJoin [c_customer_sk,customer_sk] - WholeStageCodegen (2) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #3 - WholeStageCodegen (1) - Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - WholeStageCodegen (7) - Sort [customer_sk] - InputAdapter - Exchange [customer_sk] #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow + SortMergeJoin [c_customer_sk,customer_sk] + InputAdapter + WholeStageCodegen (8) + SortMergeJoin [c_customer_sk,customer_sk] + InputAdapter + WholeStageCodegen (2) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #3 + WholeStageCodegen (1) + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (7) + Sort [customer_sk] + InputAdapter + Exchange [customer_sk] #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow + BroadcastExchange #5 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (10) + ReusedExchange [d_date_sk] #5 + InputAdapter + WholeStageCodegen (11) Sort [customer_sk] InputAdapter Exchange [customer_sk] #6 - WholeStageCodegen (9) + WholeStageCodegen (10) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk] @@ -73,7 +77,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #7 - WholeStageCodegen (11) + WholeStageCodegen (12) Project [ca_address_sk] Filter [ca_county,ca_address_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index 2dd0a13e74a74..b7f3535bd0ea2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject (55) : : +- * Filter (34) : : +- SortMergeJoin ExistenceJoin(exists#1) (33) : : :- SortMergeJoin ExistenceJoin(exists#2) (25) - : : : :- SortMergeJoin LeftSemi (17) + : : : :- * SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) @@ -127,7 +127,7 @@ Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(17) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None @@ -139,18 +139,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 8] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#17] -(21) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#17] Join condition: None -(22) Project [codegen id : 7] +(22) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] @@ -158,7 +158,7 @@ Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] -(24) Sort [codegen id : 8] +(24) Sort [codegen id : 9] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 @@ -174,18 +174,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 11] Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#21] -(29) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#20] Right keys [1]: [d_date_sk#21] Join condition: None -(30) Project [codegen id : 10] +(30) Project [codegen id : 11] Output [1]: [cs_ship_customer_sk#19] Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] @@ -193,7 +193,7 @@ Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] Input [1]: [cs_ship_customer_sk#19] Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] -(32) Sort [codegen id : 11] +(32) Sort [codegen id : 12] Input [1]: [cs_ship_customer_sk#19] Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 @@ -202,11 +202,11 @@ Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#19] Join condition: None -(34) Filter [codegen id : 13] +(34) Filter [codegen id : 14] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(35) Project [codegen id : 13] +(35) Project [codegen id : 14] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -217,14 +217,14 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 12] +(37) ColumnarToRow [codegen id : 13] Input [2]: [ca_address_sk#23, ca_county#24] -(38) Filter [codegen id : 12] +(38) Filter [codegen id : 13] Input [2]: [ca_address_sk#23, ca_county#24] Condition : (ca_county#24 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#23)) -(39) Project [codegen id : 12] +(39) Project [codegen id : 13] Output [1]: [ca_address_sk#23] Input [2]: [ca_address_sk#23, ca_county#24] @@ -232,12 +232,12 @@ Input [2]: [ca_address_sk#23, ca_county#24] Input [1]: [ca_address_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] -(41) BroadcastHashJoin [codegen id : 13] +(41) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#23] Join condition: None -(42) Project [codegen id : 13] +(42) Project [codegen id : 14] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#23] @@ -245,7 +245,7 @@ Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#23] Input [1]: [c_current_cdemo_sk#4] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#26] -(44) Sort [codegen id : 14] +(44) Sort [codegen id : 15] Input [1]: [c_current_cdemo_sk#4] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 @@ -256,10 +256,10 @@ Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 15] +(46) ColumnarToRow [codegen id : 16] Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -(47) Filter [codegen id : 15] +(47) Filter [codegen id : 16] Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Condition : isnotnull(cd_demo_sk#27) @@ -267,20 +267,20 @@ Condition : isnotnull(cd_demo_sk#27) Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#36] -(49) Sort [codegen id : 16] +(49) Sort [codegen id : 17] Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 -(50) SortMergeJoin [codegen id : 17] +(50) SortMergeJoin [codegen id : 18] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#27] Join condition: None -(51) Project [codegen id : 17] +(51) Project [codegen id : 18] Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -(52) HashAggregate [codegen id : 17] +(52) HashAggregate [codegen id : 18] Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [partial_count(1)] @@ -291,7 +291,7 @@ Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_pur Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#39] -(54) HashAggregate [codegen id : 18] +(54) HashAggregate [codegen id : 19] Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt index c10844dccad8f..a40043d68deeb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter - WholeStageCodegen (14) + WholeStageCodegen (15) Sort [c_current_cdemo_sk] InputAdapter Exchange [c_current_cdemo_sk] #2 - WholeStageCodegen (13) + WholeStageCodegen (14) Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_current_cdemo_sk,c_current_addr_sk] @@ -20,41 +20,44 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter SortMergeJoin [c_customer_sk,cs_ship_customer_sk] SortMergeJoin [c_customer_sk,ws_bill_customer_sk] - SortMergeJoin [c_customer_sk,ss_customer_sk] - WholeStageCodegen (2) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #3 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (4) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (8) + WholeStageCodegen (6) + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (2) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #3 + WholeStageCodegen (1) + Filter [c_current_addr_sk,c_current_cdemo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #4 + WholeStageCodegen (4) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (9) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #6 - WholeStageCodegen (7) + WholeStageCodegen (8) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -63,11 +66,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) + WholeStageCodegen (12) Sort [cs_ship_customer_sk] InputAdapter Exchange [cs_ship_customer_sk] #7 - WholeStageCodegen (10) + WholeStageCodegen (11) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -78,18 +81,18 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #8 - WholeStageCodegen (12) + WholeStageCodegen (13) Project [ca_address_sk] Filter [ca_county,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_county] InputAdapter - WholeStageCodegen (16) + WholeStageCodegen (17) Sort [cd_demo_sk] InputAdapter Exchange [cd_demo_sk] #9 - WholeStageCodegen (15) + WholeStageCodegen (16) Filter [cd_demo_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index 0c191216db316..8828920519371 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject (134) : +- * BroadcastHashJoin Inner BuildRight (84) : :- * Project (74) : : +- * BroadcastHashJoin Inner BuildRight (73) - : : :- SortMergeJoin LeftSemi (67) + : : :- * SortMergeJoin LeftSemi (67) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) @@ -31,7 +31,7 @@ TakeOrderedAndProject (134) : : : +- * HashAggregate (61) : : : +- Exchange (60) : : : +- * HashAggregate (59) - : : : +- SortMergeJoin LeftSemi (58) + : : : +- * SortMergeJoin LeftSemi (58) : : : :- * Sort (46) : : : : +- Exchange (45) : : : : +- * HashAggregate (44) @@ -50,7 +50,7 @@ TakeOrderedAndProject (134) : : : : : +- * ColumnarToRow (13) : : : : : +- Scan parquet default.date_dim (12) : : : : +- BroadcastExchange (39) - : : : : +- SortMergeJoin LeftSemi (38) + : : : : +- * SortMergeJoin LeftSemi (38) : : : : :- * Sort (23) : : : : : +- Exchange (22) : : : : : +- * Filter (21) @@ -87,7 +87,7 @@ TakeOrderedAndProject (134) : : +- * ColumnarToRow (69) : : +- Scan parquet default.date_dim (68) : +- BroadcastExchange (83) - : +- SortMergeJoin LeftSemi (82) + : +- * SortMergeJoin LeftSemi (82) : :- * Sort (79) : : +- Exchange (78) : : +- * Filter (77) @@ -104,7 +104,7 @@ TakeOrderedAndProject (134) : +- * BroadcastHashJoin Inner BuildRight (103) : :- * Project (101) : : +- * BroadcastHashJoin Inner BuildRight (100) - : : :- SortMergeJoin LeftSemi (98) + : : :- * SortMergeJoin LeftSemi (98) : : : :- * Sort (95) : : : : +- Exchange (94) : : : : +- * Filter (93) @@ -123,7 +123,7 @@ TakeOrderedAndProject (134) +- * BroadcastHashJoin Inner BuildRight (122) :- * Project (120) : +- * BroadcastHashJoin Inner BuildRight (119) - : :- SortMergeJoin LeftSemi (117) + : :- * SortMergeJoin LeftSemi (117) : : :- * Sort (114) : : : +- Exchange (113) : : : +- * Filter (112) @@ -165,10 +165,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -180,10 +180,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] +(10) ColumnarToRow [codegen id : 11] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 10] +(11) Filter [codegen id : 11] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Condition : isnotnull(ss_item_sk#11) @@ -209,12 +209,12 @@ Input [2]: [d_date_sk#14, d_year#15] Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 10] +(17) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#12] Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 10] +(18) Project [codegen id : 11] Output [1]: [ss_item_sk#11] Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] @@ -302,7 +302,7 @@ Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), c Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(38) SortMergeJoin +(38) SortMergeJoin [codegen id : 10] Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None @@ -311,16 +311,16 @@ Join condition: None Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(40) BroadcastHashJoin [codegen id : 10] +(40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#11] Right keys [1]: [i_item_sk#17] Join condition: None -(41) Project [codegen id : 10] +(41) Project [codegen id : 11] Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 11] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -331,7 +331,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 12] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -342,7 +342,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 12] +(46) Sort [codegen id : 13] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 @@ -354,34 +354,34 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] +(48) ColumnarToRow [codegen id : 16] Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 15] +(49) Filter [codegen id : 16] Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 15] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_sold_date_sk#38] Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 15] +(52) Project [codegen id : 16] Output [1]: [ws_item_sk#37] Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] (53) ReusedExchange [Reuses operator id: 33] Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 15] +(54) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_item_sk#37] Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 15] +(55) Project [codegen id : 16] Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] @@ -389,16 +389,16 @@ Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_categor Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 16] +(57) Sort [codegen id : 17] Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 -(58) SortMergeJoin +(58) SortMergeJoin [codegen id : 18] Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 17] +(59) HashAggregate [codegen id : 18] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -409,7 +409,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 18] +(61) HashAggregate [codegen id : 19] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -420,12 +420,12 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 19] +(64) Project [codegen id : 20] Output [1]: [i_item_sk#7 AS ss_item_sk#47] Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] @@ -433,11 +433,11 @@ Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id# Input [1]: [ss_item_sk#47] Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 20] +(66) Sort [codegen id : 21] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(67) SortMergeJoin +(67) SortMergeJoin [codegen id : 45] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -449,14 +449,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 21] +(69) ColumnarToRow [codegen id : 22] Input [3]: [d_date_sk#49, d_year#50, d_moy#51] -(70) Filter [codegen id : 21] +(70) Filter [codegen id : 22] Input [3]: [d_date_sk#49, d_year#50, d_moy#51] Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2001)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 21] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#49] Input [3]: [d_date_sk#49, d_year#50, d_moy#51] @@ -464,12 +464,12 @@ Input [3]: [d_date_sk#49, d_year#50, d_moy#51] Input [1]: [d_date_sk#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 45] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 45] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] @@ -480,10 +480,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(77) Filter [codegen id : 22] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Condition : isnotnull(i_item_sk#53) @@ -491,18 +491,18 @@ Condition : isnotnull(i_item_sk#53) Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] -(79) Sort [codegen id : 23] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 41] +(81) Sort [codegen id : 43] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(82) SortMergeJoin +(82) SortMergeJoin [codegen id : 44] Left keys [1]: [i_item_sk#53] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -511,16 +511,16 @@ Join condition: None Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 45] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#53] Join condition: None -(85) Project [codegen id : 42] +(85) Project [codegen id : 45] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(86) HashAggregate [codegen id : 42] +(86) HashAggregate [codegen id : 45] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] @@ -531,18 +531,18 @@ Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63 Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] -(88) HashAggregate [codegen id : 43] +(88) HashAggregate [codegen id : 46] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -(89) Filter [codegen id : 43] +(89) Filter [codegen id : 46] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(90) Project [codegen id : 43] +(90) Project [codegen id : 46] Output [6]: [sales#68, number_sales#69, store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] @@ -554,10 +554,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 44] +(92) ColumnarToRow [codegen id : 47] Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -(93) Filter [codegen id : 44] +(93) Filter [codegen id : 47] Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Condition : isnotnull(cs_item_sk#74) @@ -565,18 +565,18 @@ Condition : isnotnull(cs_item_sk#74) Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] -(95) Sort [codegen id : 45] +(95) Sort [codegen id : 48] Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 63] +(97) Sort [codegen id : 67] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin +(98) SortMergeJoin [codegen id : 91] Left keys [1]: [cs_item_sk#74] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -584,28 +584,28 @@ Join condition: None (99) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#79] -(100) BroadcastHashJoin [codegen id : 85] +(100) BroadcastHashJoin [codegen id : 91] Left keys [1]: [cs_sold_date_sk#77] Right keys [1]: [d_date_sk#79] Join condition: None -(101) Project [codegen id : 85] +(101) Project [codegen id : 91] Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] (102) ReusedExchange [Reuses operator id: 83] Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(103) BroadcastHashJoin [codegen id : 85] +(103) BroadcastHashJoin [codegen id : 91] Left keys [1]: [cs_item_sk#74] Right keys [1]: [i_item_sk#80] Join condition: None -(104) Project [codegen id : 85] +(104) Project [codegen id : 91] Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(105) HashAggregate [codegen id : 85] +(105) HashAggregate [codegen id : 91] Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] @@ -616,18 +616,18 @@ Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88 Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] -(107) HashAggregate [codegen id : 86] +(107) HashAggregate [codegen id : 92] Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -(108) Filter [codegen id : 86] +(108) Filter [codegen id : 92] Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(109) Project [codegen id : 86] +(109) Project [codegen id : 92] Output [6]: [sales#93, number_sales#94, catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83] Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] @@ -639,10 +639,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 87] +(111) ColumnarToRow [codegen id : 93] Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -(112) Filter [codegen id : 87] +(112) Filter [codegen id : 93] Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Condition : isnotnull(ws_item_sk#97) @@ -650,18 +650,18 @@ Condition : isnotnull(ws_item_sk#97) Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] -(114) Sort [codegen id : 88] +(114) Sort [codegen id : 94] Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(116) Sort [codegen id : 106] +(116) Sort [codegen id : 113] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(117) SortMergeJoin +(117) SortMergeJoin [codegen id : 137] Left keys [1]: [ws_item_sk#97] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -669,28 +669,28 @@ Join condition: None (118) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#102] -(119) BroadcastHashJoin [codegen id : 128] +(119) BroadcastHashJoin [codegen id : 137] Left keys [1]: [ws_sold_date_sk#100] Right keys [1]: [d_date_sk#102] Join condition: None -(120) Project [codegen id : 128] +(120) Project [codegen id : 137] Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] (121) ReusedExchange [Reuses operator id: 83] Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(122) BroadcastHashJoin [codegen id : 128] +(122) BroadcastHashJoin [codegen id : 137] Left keys [1]: [ws_item_sk#97] Right keys [1]: [i_item_sk#103] Join condition: None -(123) Project [codegen id : 128] +(123) Project [codegen id : 137] Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(124) HashAggregate [codegen id : 128] +(124) HashAggregate [codegen id : 137] Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] @@ -701,28 +701,28 @@ Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpt Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] -(126) HashAggregate [codegen id : 129] +(126) HashAggregate [codegen id : 138] Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -(127) Filter [codegen id : 129] +(127) Filter [codegen id : 138] Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(128) Project [codegen id : 129] +(128) Project [codegen id : 138] Output [6]: [sales#116, number_sales#117, web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106] Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] (129) Union -(130) Expand [codegen id : 130] +(130) Expand [codegen id : 139] Input [6]: [sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: [ArrayBuffer(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 0), ArrayBuffer(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, null, 1), ArrayBuffer(sales#68, number_sales#69, channel#73, i_brand_id#54, null, null, 3), ArrayBuffer(sales#68, number_sales#69, channel#73, null, null, null, 7), ArrayBuffer(sales#68, number_sales#69, null, null, null, null, 15)], [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] -(131) HashAggregate [codegen id : 130] +(131) HashAggregate [codegen id : 139] Input [7]: [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] @@ -733,7 +733,7 @@ Results [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sp Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] Arguments: hashpartitioning(channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, 5), ENSURE_REQUIREMENTS, [id=#131] -(133) HashAggregate [codegen id : 131] +(133) HashAggregate [codegen id : 140] Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] Functions [2]: [sum(sales#68), sum(number_sales#69)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 2134091c43b82..bf3b205bb8c58 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (131) + WholeStageCodegen (140) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (130) + WholeStageCodegen (139) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (43) + WholeStageCodegen (46) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -57,14 +57,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (42) + WholeStageCodegen (45) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk,ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [ss_item_sk,ss_item_sk] + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter WholeStageCodegen (2) Sort [ss_item_sk] InputAdapter @@ -76,11 +76,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - WholeStageCodegen (20) + InputAdapter + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -89,23 +90,23 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -127,44 +128,48 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #11 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #10 InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + BroadcastExchange #14 + WholeStageCodegen (7) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -180,7 +185,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter BroadcastExchange #4 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -188,46 +193,50 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #16 - SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #17 - WholeStageCodegen (22) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk] #18 - WholeStageCodegen (86) + WholeStageCodegen (44) + SortMergeJoin [i_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (24) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #17 + WholeStageCodegen (23) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (43) + Sort [ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk] #18 + WholeStageCodegen (92) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (85) + WholeStageCodegen (91) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_item_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (45) + SortMergeJoin [cs_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (48) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #22 - WholeStageCodegen (44) + WholeStageCodegen (47) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (63) + InputAdapter + WholeStageCodegen (67) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 @@ -235,32 +244,33 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (129) + WholeStageCodegen (138) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (128) + WholeStageCodegen (137) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk,ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (88) + SortMergeJoin [ws_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (94) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #24 - WholeStageCodegen (87) + WholeStageCodegen (93) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (106) + InputAdapter + WholeStageCodegen (113) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index fab231dc80a6c..d0bdd41048367 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (116) : +- * BroadcastHashJoin Inner BuildRight (84) : :- * Project (74) : : +- * BroadcastHashJoin Inner BuildRight (73) - : : :- SortMergeJoin LeftSemi (67) + : : :- * SortMergeJoin LeftSemi (67) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) @@ -27,7 +27,7 @@ TakeOrderedAndProject (116) : : : +- * HashAggregate (61) : : : +- Exchange (60) : : : +- * HashAggregate (59) - : : : +- SortMergeJoin LeftSemi (58) + : : : +- * SortMergeJoin LeftSemi (58) : : : :- * Sort (46) : : : : +- Exchange (45) : : : : +- * HashAggregate (44) @@ -46,7 +46,7 @@ TakeOrderedAndProject (116) : : : : : +- * ColumnarToRow (13) : : : : : +- Scan parquet default.date_dim (12) : : : : +- BroadcastExchange (39) - : : : : +- SortMergeJoin LeftSemi (38) + : : : : +- * SortMergeJoin LeftSemi (38) : : : : :- * Sort (23) : : : : : +- Exchange (22) : : : : : +- * Filter (21) @@ -83,7 +83,7 @@ TakeOrderedAndProject (116) : : +- * ColumnarToRow (69) : : +- Scan parquet default.date_dim (68) : +- BroadcastExchange (83) - : +- SortMergeJoin LeftSemi (82) + : +- * SortMergeJoin LeftSemi (82) : :- * Sort (79) : : +- Exchange (78) : : +- * Filter (77) @@ -101,7 +101,7 @@ TakeOrderedAndProject (116) +- * BroadcastHashJoin Inner BuildRight (107) :- * Project (105) : +- * BroadcastHashJoin Inner BuildRight (104) - : :- SortMergeJoin LeftSemi (98) + : :- * SortMergeJoin LeftSemi (98) : : :- * Sort (95) : : : +- Exchange (94) : : : +- * Filter (93) @@ -147,10 +147,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -162,10 +162,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] +(10) ColumnarToRow [codegen id : 11] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 10] +(11) Filter [codegen id : 11] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Condition : isnotnull(ss_item_sk#11) @@ -191,12 +191,12 @@ Input [2]: [d_date_sk#14, d_year#15] Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 10] +(17) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#12] Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 10] +(18) Project [codegen id : 11] Output [1]: [ss_item_sk#11] Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] @@ -284,7 +284,7 @@ Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), c Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(38) SortMergeJoin +(38) SortMergeJoin [codegen id : 10] Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None @@ -293,16 +293,16 @@ Join condition: None Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(40) BroadcastHashJoin [codegen id : 10] +(40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#11] Right keys [1]: [i_item_sk#17] Join condition: None -(41) Project [codegen id : 10] +(41) Project [codegen id : 11] Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 11] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -313,7 +313,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 12] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -324,7 +324,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 12] +(46) Sort [codegen id : 13] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 @@ -336,34 +336,34 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] +(48) ColumnarToRow [codegen id : 16] Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 15] +(49) Filter [codegen id : 16] Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 15] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_sold_date_sk#38] Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 15] +(52) Project [codegen id : 16] Output [1]: [ws_item_sk#37] Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] (53) ReusedExchange [Reuses operator id: 33] Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 15] +(54) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_item_sk#37] Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 15] +(55) Project [codegen id : 16] Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] @@ -371,16 +371,16 @@ Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_categor Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 16] +(57) Sort [codegen id : 17] Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 -(58) SortMergeJoin +(58) SortMergeJoin [codegen id : 18] Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 17] +(59) HashAggregate [codegen id : 18] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -391,7 +391,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 18] +(61) HashAggregate [codegen id : 19] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -402,12 +402,12 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 19] +(64) Project [codegen id : 20] Output [1]: [i_item_sk#7 AS ss_item_sk#47] Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] @@ -415,11 +415,11 @@ Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id# Input [1]: [ss_item_sk#47] Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 20] +(66) Sort [codegen id : 21] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(67) SortMergeJoin +(67) SortMergeJoin [codegen id : 45] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -431,14 +431,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 21] +(69) ColumnarToRow [codegen id : 22] Input [2]: [d_date_sk#49, d_week_seq#50] -(70) Filter [codegen id : 21] +(70) Filter [codegen id : 22] Input [2]: [d_date_sk#49, d_week_seq#50] Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 21] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#49] Input [2]: [d_date_sk#49, d_week_seq#50] @@ -446,12 +446,12 @@ Input [2]: [d_date_sk#49, d_week_seq#50] Input [1]: [d_date_sk#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 45] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 45] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] @@ -462,10 +462,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(77) Filter [codegen id : 22] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) @@ -473,18 +473,18 @@ Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnu Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] -(79) Sort [codegen id : 23] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 41] +(81) Sort [codegen id : 43] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(82) SortMergeJoin +(82) SortMergeJoin [codegen id : 44] Left keys [1]: [i_item_sk#54] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -493,16 +493,16 @@ Join condition: None Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 45] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#54] Join condition: None -(85) Project [codegen id : 42] +(85) Project [codegen id : 45] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(86) HashAggregate [codegen id : 42] +(86) HashAggregate [codegen id : 45] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] @@ -513,18 +513,18 @@ Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64 Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] -(88) HashAggregate [codegen id : 86] +(88) HashAggregate [codegen id : 92] Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -(89) Filter [codegen id : 86] +(89) Filter [codegen id : 92] Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(90) Project [codegen id : 86] +(90) Project [codegen id : 92] Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] @@ -536,10 +536,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 43] +(92) ColumnarToRow [codegen id : 46] Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -(93) Filter [codegen id : 43] +(93) Filter [codegen id : 46] Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Condition : isnotnull(ss_item_sk#75) @@ -547,18 +547,18 @@ Condition : isnotnull(ss_item_sk#75) Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] -(95) Sort [codegen id : 44] +(95) Sort [codegen id : 47] Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 62] +(97) Sort [codegen id : 66] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin +(98) SortMergeJoin [codegen id : 90] Left keys [1]: [ss_item_sk#75] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -570,14 +570,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 63] +(100) ColumnarToRow [codegen id : 67] Input [2]: [d_date_sk#81, d_week_seq#82] -(101) Filter [codegen id : 63] +(101) Filter [codegen id : 67] Input [2]: [d_date_sk#81, d_week_seq#82] Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) -(102) Project [codegen id : 63] +(102) Project [codegen id : 67] Output [1]: [d_date_sk#81] Input [2]: [d_date_sk#81, d_week_seq#82] @@ -585,28 +585,28 @@ Input [2]: [d_date_sk#81, d_week_seq#82] Input [1]: [d_date_sk#81] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] -(104) BroadcastHashJoin [codegen id : 84] +(104) BroadcastHashJoin [codegen id : 90] Left keys [1]: [ss_sold_date_sk#78] Right keys [1]: [d_date_sk#81] Join condition: None -(105) Project [codegen id : 84] +(105) Project [codegen id : 90] Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] (106) ReusedExchange [Reuses operator id: 83] Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(107) BroadcastHashJoin [codegen id : 84] +(107) BroadcastHashJoin [codegen id : 90] Left keys [1]: [ss_item_sk#75] Right keys [1]: [i_item_sk#86] Join condition: None -(108) Project [codegen id : 84] +(108) Project [codegen id : 90] Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(109) HashAggregate [codegen id : 84] +(109) HashAggregate [codegen id : 90] Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] @@ -617,18 +617,18 @@ Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94 Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] -(111) HashAggregate [codegen id : 85] +(111) HashAggregate [codegen id : 91] Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -(112) Filter [codegen id : 85] +(112) Filter [codegen id : 91] Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(113) Project [codegen id : 85] +(113) Project [codegen id : 91] Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] @@ -636,7 +636,7 @@ Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sal Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] -(115) BroadcastHashJoin [codegen id : 86] +(115) BroadcastHashJoin [codegen id : 92] Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index 1b41abbc97b06..6a8fcada7a61b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (86) + WholeStageCodegen (92) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,14 +49,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (42) + WholeStageCodegen (45) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk,ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [ss_item_sk,ss_item_sk] + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter WholeStageCodegen (2) Sort [ss_item_sk] InputAdapter @@ -68,11 +68,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (20) + InputAdapter + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,23 +82,23 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -119,44 +120,48 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #9 InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -172,7 +177,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -187,55 +192,59 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #15 - SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #16 - WholeStageCodegen (22) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk] #17 + WholeStageCodegen (44) + SortMergeJoin [i_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (24) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #16 + WholeStageCodegen (23) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (43) + Sort [ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (85) + WholeStageCodegen (91) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (84) + WholeStageCodegen (90) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk,ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (44) + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (47) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (43) + WholeStageCodegen (46) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (62) + InputAdapter + WholeStageCodegen (66) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (63) + WholeStageCodegen (67) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 7604ea1e786d8..f9ab964739273 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -13,7 +13,7 @@ : : +- * BroadcastHashJoin Inner BuildRight (25) : : :- SortMergeJoin LeftAnti (19) : : : :- * Project (13) - : : : : +- SortMergeJoin LeftSemi (12) + : : : : +- * SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) @@ -94,7 +94,7 @@ Arguments: hashpartitioning(cs_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13 Input [2]: [cs_warehouse_sk#10, cs_order_number#11] Arguments: [cs_order_number#11 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin +(12) SortMergeJoin [codegen id : 5] Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#11] Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#10) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt index 393a5baec4400..489200f5201eb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt @@ -17,8 +17,8 @@ WholeStageCodegen (12) SortMergeJoin [cs_order_number,cr_order_number] WholeStageCodegen (5) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - InputAdapter - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter WholeStageCodegen (2) Sort [cs_order_number] InputAdapter @@ -29,6 +29,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + InputAdapter WholeStageCodegen (4) Sort [cs_order_number] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 683d83235cce5..647824d3a9d75 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -13,7 +13,7 @@ : : +- * BroadcastHashJoin Inner BuildRight (25) : : :- SortMergeJoin LeftAnti (19) : : : :- * Project (13) - : : : : +- SortMergeJoin LeftSemi (12) + : : : : +- * SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) @@ -94,7 +94,7 @@ Arguments: hashpartitioning(cs_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13 Input [2]: [cs_warehouse_sk#10, cs_order_number#11] Arguments: [cs_order_number#11 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin +(12) SortMergeJoin [codegen id : 5] Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#11] Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#10) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 5edd1426dd8f1..c7ead9a46797a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -17,8 +17,8 @@ WholeStageCodegen (12) SortMergeJoin [cs_order_number,cr_order_number] WholeStageCodegen (5) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - InputAdapter - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter WholeStageCodegen (2) Sort [cs_order_number] InputAdapter @@ -29,6 +29,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + InputAdapter WholeStageCodegen (4) Sort [cs_order_number] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt index 95c669b74adfc..c61606d124ccb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt @@ -6,11 +6,11 @@ :- * Project (59) : +- * BroadcastHashJoin Inner BuildRight (58) : :- * Project (52) - : : +- SortMergeJoin LeftSemi (51) + : : +- * SortMergeJoin LeftSemi (51) : : :- * Sort (32) : : : +- Exchange (31) : : : +- * Project (30) - : : : +- SortMergeJoin LeftSemi (29) + : : : +- * SortMergeJoin LeftSemi (29) : : : :- * Sort (4) : : : : +- Exchange (3) : : : : +- * ColumnarToRow (2) @@ -65,11 +65,11 @@ +- * Project (94) +- * BroadcastHashJoin Inner BuildRight (93) :- * Project (91) - : +- SortMergeJoin LeftSemi (90) + : +- * SortMergeJoin LeftSemi (90) : :- * Sort (78) : : +- Exchange (77) : : +- * Project (76) - : : +- SortMergeJoin LeftSemi (75) + : : +- * SortMergeJoin LeftSemi (75) : : :- * Sort (63) : : : +- Exchange (62) : : : +- * ColumnarToRow (61) @@ -228,7 +228,7 @@ Input [2]: [item_sk#23, count(1)#24] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin +(29) SortMergeJoin [codegen id : 9] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#23] Join condition: None @@ -328,7 +328,7 @@ Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_ Input [1]: [c_customer_sk#31] Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 -(51) SortMergeJoin +(51) SortMergeJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#31] Join condition: None @@ -435,7 +435,7 @@ Input [2]: [item_sk#23, count(1)#62] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 -(75) SortMergeJoin +(75) SortMergeJoin [codegen id : 26] Left keys [1]: [ws_item_sk#48] Right keys [1]: [item_sk#23] Join condition: None @@ -501,7 +501,7 @@ Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_ Input [1]: [c_customer_sk#67] Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin +(90) SortMergeJoin [codegen id : 34] Left keys [1]: [ws_bill_customer_sk#49] Right keys [1]: [c_customer_sk#67] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt index 9144785933616..0e8b0977ae088 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt @@ -10,16 +10,16 @@ WholeStageCodegen (36) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_quantity,cs_list_price,cs_sold_date_sk] - InputAdapter - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter WholeStageCodegen (10) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 WholeStageCodegen (9) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - InputAdapter - SortMergeJoin [cs_item_sk,item_sk] + SortMergeJoin [cs_item_sk,item_sk] + InputAdapter WholeStageCodegen (2) Sort [cs_item_sk] InputAdapter @@ -30,6 +30,7 @@ WholeStageCodegen (36) Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 + InputAdapter WholeStageCodegen (8) Sort [item_sk] Project [item_sk] @@ -70,6 +71,7 @@ WholeStageCodegen (36) ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] + InputAdapter WholeStageCodegen (15) Sort [c_customer_sk] Project [c_customer_sk] @@ -154,16 +156,16 @@ WholeStageCodegen (36) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_quantity,ws_list_price,ws_sold_date_sk] - InputAdapter - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter WholeStageCodegen (27) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #14 WholeStageCodegen (26) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - InputAdapter - SortMergeJoin [ws_item_sk,item_sk] + SortMergeJoin [ws_item_sk,item_sk] + InputAdapter WholeStageCodegen (19) Sort [ws_item_sk] InputAdapter @@ -173,6 +175,7 @@ WholeStageCodegen (36) InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 + InputAdapter WholeStageCodegen (25) Sort [item_sk] Project [item_sk] @@ -191,6 +194,7 @@ WholeStageCodegen (36) Sort [i_item_sk] InputAdapter ReusedExchange [i_item_sk,i_item_desc] #7 + InputAdapter WholeStageCodegen (32) Sort [c_customer_sk] Project [c_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 176463544e0f6..2bce3e4d9bb60 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -6,7 +6,7 @@ :- * Project (53) : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (46) - : : +- SortMergeJoin LeftSemi (45) + : : +- * SortMergeJoin LeftSemi (45) : : :- * Sort (28) : : : +- Exchange (27) : : : +- * Project (26) @@ -59,7 +59,7 @@ +- * Project (70) +- * BroadcastHashJoin Inner BuildRight (69) :- * Project (67) - : +- SortMergeJoin LeftSemi (66) + : +- * SortMergeJoin LeftSemi (66) : :- * Sort (60) : : +- Exchange (59) : : +- * Project (58) @@ -280,7 +280,7 @@ Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_ Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(45) SortMergeJoin +(45) SortMergeJoin [codegen id : 11] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#30] Join condition: None @@ -372,7 +372,7 @@ Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_ Input [1]: [c_customer_sk#54] Arguments: [c_customer_sk#54 ASC NULLS FIRST], false, 0 -(66) SortMergeJoin +(66) SortMergeJoin [codegen id : 22] Left keys [1]: [ws_bill_customer_sk#49] Right keys [1]: [c_customer_sk#54] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index d89287382c5c8..697c1f497c399 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -10,8 +10,8 @@ WholeStageCodegen (24) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_quantity,cs_list_price,cs_sold_date_sk] - InputAdapter - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter WholeStageCodegen (6) Sort [cs_bill_customer_sk] InputAdapter @@ -59,6 +59,7 @@ WholeStageCodegen (24) ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] + InputAdapter WholeStageCodegen (9) Sort [c_customer_sk] Project [c_customer_sk] @@ -131,8 +132,8 @@ WholeStageCodegen (24) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_quantity,ws_list_price,ws_sold_date_sk] - InputAdapter - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter WholeStageCodegen (17) Sort [ws_bill_customer_sk] InputAdapter @@ -146,6 +147,7 @@ WholeStageCodegen (24) ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [item_sk] #15 + InputAdapter WholeStageCodegen (20) Sort [c_customer_sk] Project [c_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index 690dd1b6099e5..c5b8ad4607553 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject (137) : +- * SortMergeJoin Inner (77) : :- * Project (59) : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- SortMergeJoin LeftSemi (52) + : : :- * SortMergeJoin LeftSemi (52) : : : :- * Sort (33) : : : : +- Exchange (32) : : : : +- * Project (31) - : : : : +- SortMergeJoin LeftSemi (30) + : : : : +- * SortMergeJoin LeftSemi (30) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) @@ -65,7 +65,7 @@ TakeOrderedAndProject (137) : : +- * Filter (55) : : +- * ColumnarToRow (54) : : +- Scan parquet default.date_dim (53) - : +- SortMergeJoin LeftSemi (76) + : +- * SortMergeJoin LeftSemi (76) : :- * Sort (64) : : +- Exchange (63) : : +- * Filter (62) @@ -89,11 +89,11 @@ TakeOrderedAndProject (137) +- * SortMergeJoin Inner (131) :- * Project (116) : +- * BroadcastHashJoin Inner BuildRight (115) - : :- SortMergeJoin LeftSemi (113) + : :- * SortMergeJoin LeftSemi (113) : : :- * Sort (101) : : : +- Exchange (100) : : : +- * Project (99) - : : : +- SortMergeJoin LeftSemi (98) + : : : +- * SortMergeJoin LeftSemi (98) : : : :- * Sort (86) : : : : +- Exchange (85) : : : : +- * Filter (84) @@ -122,7 +122,7 @@ TakeOrderedAndProject (137) : : +- * Sort (105) : : +- ReusedExchange (104) : +- ReusedExchange (114) - +- SortMergeJoin LeftSemi (130) + +- * SortMergeJoin LeftSemi (130) :- * Sort (118) : +- ReusedExchange (117) +- * Sort (129) @@ -272,7 +272,7 @@ Input [2]: [item_sk#23, count(1)#24] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 -(30) SortMergeJoin +(30) SortMergeJoin [codegen id : 9] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#23] Join condition: None @@ -372,7 +372,7 @@ Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_ Input [1]: [c_customer_sk#31] Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 -(52) SortMergeJoin +(52) SortMergeJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#31] Join condition: None @@ -479,21 +479,21 @@ Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_ Input [1]: [c_customer_sk#31] Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 -(76) SortMergeJoin +(76) SortMergeJoin [codegen id : 25] Left keys [1]: [c_customer_sk#47] Right keys [1]: [c_customer_sk#31] Join condition: None -(77) SortMergeJoin [codegen id : 25] +(77) SortMergeJoin [codegen id : 26] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#47] Join condition: None -(78) Project [codegen id : 25] +(78) Project [codegen id : 26] Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#47, c_first_name#48, c_last_name#49] -(79) HashAggregate [codegen id : 25] +(79) HashAggregate [codegen id : 26] Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] Keys [2]: [c_last_name#49, c_first_name#48] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] @@ -504,7 +504,7 @@ Results [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] Arguments: hashpartitioning(c_last_name#49, c_first_name#48, 5), ENSURE_REQUIREMENTS, [id=#55] -(81) HashAggregate [codegen id : 26] +(81) HashAggregate [codegen id : 27] Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] Keys [2]: [c_last_name#49, c_first_name#48] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] @@ -519,10 +519,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(83) ColumnarToRow [codegen id : 27] +(83) ColumnarToRow [codegen id : 28] Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -(84) Filter [codegen id : 27] +(84) Filter [codegen id : 28] Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] Condition : isnotnull(ws_bill_customer_sk#59) @@ -530,65 +530,65 @@ Condition : isnotnull(ws_bill_customer_sk#59) Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] Arguments: hashpartitioning(ws_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#63] -(86) Sort [codegen id : 28] +(86) Sort [codegen id : 29] Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] Arguments: [ws_item_sk#58 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: unknown] Output [2]: [ss_item_sk#64, d_date#65] -(88) Sort [codegen id : 31] +(88) Sort [codegen id : 32] Input [2]: [ss_item_sk#64, d_date#65] Arguments: [ss_item_sk#64 ASC NULLS FIRST], false, 0 (89) ReusedExchange [Reuses operator id: 21] Output [2]: [i_item_sk#66, i_item_desc#67] -(90) Sort [codegen id : 33] +(90) Sort [codegen id : 34] Input [2]: [i_item_sk#66, i_item_desc#67] Arguments: [i_item_sk#66 ASC NULLS FIRST], false, 0 -(91) SortMergeJoin [codegen id : 34] +(91) SortMergeJoin [codegen id : 35] Left keys [1]: [ss_item_sk#64] Right keys [1]: [i_item_sk#66] Join condition: None -(92) Project [codegen id : 34] +(92) Project [codegen id : 35] Output [3]: [d_date#65, i_item_sk#66, substr(i_item_desc#67, 1, 30) AS _groupingexpression#68] Input [4]: [ss_item_sk#64, d_date#65, i_item_sk#66, i_item_desc#67] -(93) HashAggregate [codegen id : 34] +(93) HashAggregate [codegen id : 35] Input [3]: [d_date#65, i_item_sk#66, _groupingexpression#68] Keys [3]: [_groupingexpression#68, i_item_sk#66, d_date#65] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#69] Results [4]: [_groupingexpression#68, i_item_sk#66, d_date#65, count#70] -(94) HashAggregate [codegen id : 34] +(94) HashAggregate [codegen id : 35] Input [4]: [_groupingexpression#68, i_item_sk#66, d_date#65, count#70] Keys [3]: [_groupingexpression#68, i_item_sk#66, d_date#65] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#71] Results [2]: [i_item_sk#66 AS item_sk#23, count(1)#71 AS count(1)#72] -(95) Filter [codegen id : 34] +(95) Filter [codegen id : 35] Input [2]: [item_sk#23, count(1)#72] Condition : (count(1)#72 > 4) -(96) Project [codegen id : 34] +(96) Project [codegen id : 35] Output [1]: [item_sk#23] Input [2]: [item_sk#23, count(1)#72] -(97) Sort [codegen id : 34] +(97) Sort [codegen id : 35] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin +(98) SortMergeJoin [codegen id : 36] Left keys [1]: [ws_item_sk#58] Right keys [1]: [item_sk#23] Join condition: None -(99) Project [codegen id : 35] +(99) Project [codegen id : 36] Output [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] @@ -596,60 +596,60 @@ Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#73] -(101) Sort [codegen id : 36] +(101) Sort [codegen id : 37] Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 (102) ReusedExchange [Reuses operator id: 38] Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -(103) Sort [codegen id : 38] +(103) Sort [codegen id : 39] Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 (104) ReusedExchange [Reuses operator id: 43] Output [1]: [c_customer_sk#77] -(105) Sort [codegen id : 40] +(105) Sort [codegen id : 41] Input [1]: [c_customer_sk#77] Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 -(106) SortMergeJoin [codegen id : 41] +(106) SortMergeJoin [codegen id : 42] Left keys [1]: [ss_customer_sk#74] Right keys [1]: [c_customer_sk#77] Join condition: None -(107) Project [codegen id : 41] +(107) Project [codegen id : 42] Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -(108) HashAggregate [codegen id : 41] +(108) HashAggregate [codegen id : 42] Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] Keys [1]: [c_customer_sk#77] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#78, isEmpty#79] Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] -(109) HashAggregate [codegen id : 41] +(109) HashAggregate [codegen id : 42] Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] Keys [1]: [c_customer_sk#77] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] -(110) Filter [codegen id : 41] +(110) Filter [codegen id : 42] Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) -(111) Project [codegen id : 41] +(111) Project [codegen id : 42] Output [1]: [c_customer_sk#77] Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] -(112) Sort [codegen id : 41] +(112) Sort [codegen id : 42] Input [1]: [c_customer_sk#77] Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 -(113) SortMergeJoin +(113) SortMergeJoin [codegen id : 44] Left keys [1]: [ws_bill_customer_sk#59] Right keys [1]: [c_customer_sk#77] Join condition: None @@ -657,86 +657,86 @@ Join condition: None (114) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#84] -(115) BroadcastHashJoin [codegen id : 43] +(115) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#62] Right keys [1]: [d_date_sk#84] Join condition: None -(116) Project [codegen id : 43] +(116) Project [codegen id : 44] Output [3]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61] Input [5]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62, d_date_sk#84] (117) ReusedExchange [Reuses operator id: 63] Output [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] -(118) Sort [codegen id : 45] +(118) Sort [codegen id : 46] Input [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] Arguments: [c_customer_sk#85 ASC NULLS FIRST], false, 0 (119) ReusedExchange [Reuses operator id: 38] Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -(120) Sort [codegen id : 47] +(120) Sort [codegen id : 48] Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 (121) ReusedExchange [Reuses operator id: 43] Output [1]: [c_customer_sk#77] -(122) Sort [codegen id : 49] +(122) Sort [codegen id : 50] Input [1]: [c_customer_sk#77] Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 -(123) SortMergeJoin [codegen id : 50] +(123) SortMergeJoin [codegen id : 51] Left keys [1]: [ss_customer_sk#74] Right keys [1]: [c_customer_sk#77] Join condition: None -(124) Project [codegen id : 50] +(124) Project [codegen id : 51] Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -(125) HashAggregate [codegen id : 50] +(125) HashAggregate [codegen id : 51] Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] Keys [1]: [c_customer_sk#77] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#78, isEmpty#79] Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] -(126) HashAggregate [codegen id : 50] +(126) HashAggregate [codegen id : 51] Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] Keys [1]: [c_customer_sk#77] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] -(127) Filter [codegen id : 50] +(127) Filter [codegen id : 51] Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) -(128) Project [codegen id : 50] +(128) Project [codegen id : 51] Output [1]: [c_customer_sk#77] Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] -(129) Sort [codegen id : 50] +(129) Sort [codegen id : 51] Input [1]: [c_customer_sk#77] Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 -(130) SortMergeJoin +(130) SortMergeJoin [codegen id : 52] Left keys [1]: [c_customer_sk#85] Right keys [1]: [c_customer_sk#77] Join condition: None -(131) SortMergeJoin [codegen id : 51] +(131) SortMergeJoin [codegen id : 53] Left keys [1]: [ws_bill_customer_sk#59] Right keys [1]: [c_customer_sk#85] Join condition: None -(132) Project [codegen id : 51] +(132) Project [codegen id : 53] Output [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] Input [6]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, c_customer_sk#85, c_first_name#86, c_last_name#87] -(133) HashAggregate [codegen id : 51] +(133) HashAggregate [codegen id : 53] Input [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] Keys [2]: [c_last_name#87, c_first_name#86] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] @@ -747,7 +747,7 @@ Results [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] Arguments: hashpartitioning(c_last_name#87, c_first_name#86, 5), ENSURE_REQUIREMENTS, [id=#92] -(135) HashAggregate [codegen id : 52] +(135) HashAggregate [codegen id : 54] Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] Keys [2]: [c_last_name#87, c_first_name#86] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt index fec70a32589c7..98848b4ed0676 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (26) + WholeStageCodegen (27) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (25) + WholeStageCodegen (26) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] @@ -12,16 +12,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (17) Project [cs_bill_customer_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter WholeStageCodegen (10) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 WholeStageCodegen (9) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - InputAdapter - SortMergeJoin [cs_item_sk,item_sk] + SortMergeJoin [cs_item_sk,item_sk] + InputAdapter WholeStageCodegen (2) Sort [cs_item_sk] InputAdapter @@ -33,6 +33,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 + InputAdapter WholeStageCodegen (8) Sort [item_sk] Project [item_sk] @@ -73,6 +74,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] + InputAdapter WholeStageCodegen (15) Sort [c_customer_sk] Project [c_customer_sk] @@ -154,68 +156,72 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - SortMergeJoin [c_customer_sk,c_customer_sk] - WholeStageCodegen (19) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #14 - WholeStageCodegen (18) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (24) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (21) - Sort [ss_customer_sk] - InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 - InputAdapter - WholeStageCodegen (23) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk] #9 - WholeStageCodegen (52) + WholeStageCodegen (25) + SortMergeJoin [c_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (19) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #14 + WholeStageCodegen (18) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + WholeStageCodegen (24) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (21) + Sort [ss_customer_sk] + InputAdapter + ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 + InputAdapter + WholeStageCodegen (23) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk] #9 + WholeStageCodegen (54) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #15 - WholeStageCodegen (51) + WholeStageCodegen (53) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (44) Project [ws_bill_customer_sk,ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - WholeStageCodegen (36) + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (37) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #16 - WholeStageCodegen (35) + WholeStageCodegen (36) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - InputAdapter - SortMergeJoin [ws_item_sk,item_sk] - WholeStageCodegen (28) + SortMergeJoin [ws_item_sk,item_sk] + InputAdapter + WholeStageCodegen (29) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #17 - WholeStageCodegen (27) + WholeStageCodegen (28) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (34) + InputAdapter + WholeStageCodegen (35) Sort [item_sk] Project [item_sk] Filter [count(1)] @@ -224,16 +230,17 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Project [d_date,i_item_sk,i_item_desc] SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (31) + WholeStageCodegen (32) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk,d_date] #18 InputAdapter - WholeStageCodegen (33) + WholeStageCodegen (34) Sort [i_item_sk] InputAdapter ReusedExchange [i_item_sk,i_item_desc] #7 - WholeStageCodegen (41) + InputAdapter + WholeStageCodegen (42) Sort [c_customer_sk] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -243,39 +250,42 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (38) + WholeStageCodegen (39) Sort [ss_customer_sk] InputAdapter ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 InputAdapter - WholeStageCodegen (40) + WholeStageCodegen (41) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - SortMergeJoin [c_customer_sk,c_customer_sk] - WholeStageCodegen (45) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 - WholeStageCodegen (50) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (47) - Sort [ss_customer_sk] - InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 - InputAdapter - WholeStageCodegen (49) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk] #9 + WholeStageCodegen (52) + SortMergeJoin [c_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (46) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 + InputAdapter + WholeStageCodegen (51) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (48) + Sort [ss_customer_sk] + InputAdapter + ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 + InputAdapter + WholeStageCodegen (50) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 5e06310b2ac44..a7d797dbb545e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (103) : +- * BroadcastHashJoin Inner BuildRight (66) : :- * Project (60) : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- SortMergeJoin LeftSemi (46) + : : :- * SortMergeJoin LeftSemi (46) : : : :- * Sort (29) : : : : +- Exchange (28) : : : : +- * Project (27) @@ -55,7 +55,7 @@ TakeOrderedAndProject (103) : : : +- * ColumnarToRow (35) : : : +- Scan parquet default.customer (34) : : +- BroadcastExchange (58) - : : +- SortMergeJoin LeftSemi (57) + : : +- * SortMergeJoin LeftSemi (57) : : :- * Sort (51) : : : +- Exchange (50) : : : +- * Filter (49) @@ -78,7 +78,7 @@ TakeOrderedAndProject (103) +- * BroadcastHashJoin Inner BuildRight (97) :- * Project (95) : +- * BroadcastHashJoin Inner BuildRight (94) - : :- SortMergeJoin LeftSemi (84) + : :- * SortMergeJoin LeftSemi (84) : : :- * Sort (78) : : : +- Exchange (77) : : : +- * Project (76) @@ -93,7 +93,7 @@ TakeOrderedAndProject (103) : : +- * HashAggregate (80) : : +- ReusedExchange (79) : +- BroadcastExchange (93) - : +- SortMergeJoin LeftSemi (92) + : +- * SortMergeJoin LeftSemi (92) : :- * Sort (86) : : +- ReusedExchange (85) : +- * Sort (91) @@ -314,7 +314,7 @@ Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_ Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(46) SortMergeJoin +(46) SortMergeJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#30] Join condition: None @@ -363,7 +363,7 @@ Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_ Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(57) SortMergeJoin +(57) SortMergeJoin [codegen id : 15] Left keys [1]: [c_customer_sk#43] Right keys [1]: [c_customer_sk#30] Join condition: None @@ -372,12 +372,12 @@ Join condition: None Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] -(59) BroadcastHashJoin [codegen id : 16] +(59) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#43] Join condition: None -(60) Project [codegen id : 16] +(60) Project [codegen id : 17] Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45] Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#43, c_first_name#44, c_last_name#45] @@ -388,14 +388,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(62) ColumnarToRow [codegen id : 15] +(62) ColumnarToRow [codegen id : 16] Input [3]: [d_date_sk#48, d_year#49, d_moy#50] -(63) Filter [codegen id : 15] +(63) Filter [codegen id : 16] Input [3]: [d_date_sk#48, d_year#49, d_moy#50] Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#48)) -(64) Project [codegen id : 15] +(64) Project [codegen id : 16] Output [1]: [d_date_sk#48] Input [3]: [d_date_sk#48, d_year#49, d_moy#50] @@ -403,16 +403,16 @@ Input [3]: [d_date_sk#48, d_year#49, d_moy#50] Input [1]: [d_date_sk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] -(66) BroadcastHashJoin [codegen id : 16] +(66) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#48] Join condition: None -(67) Project [codegen id : 16] +(67) Project [codegen id : 17] Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45, d_date_sk#48] -(68) HashAggregate [codegen id : 16] +(68) HashAggregate [codegen id : 17] Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] Keys [2]: [c_last_name#45, c_first_name#44] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] @@ -423,7 +423,7 @@ Results [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] Arguments: hashpartitioning(c_last_name#45, c_first_name#44, 5), ENSURE_REQUIREMENTS, [id=#56] -(70) HashAggregate [codegen id : 17] +(70) HashAggregate [codegen id : 18] Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] Keys [2]: [c_last_name#45, c_first_name#44] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] @@ -438,22 +438,22 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(72) ColumnarToRow [codegen id : 22] +(72) ColumnarToRow [codegen id : 23] Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -(73) Filter [codegen id : 22] +(73) Filter [codegen id : 23] Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] Condition : isnotnull(ws_bill_customer_sk#60) (74) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] -(75) BroadcastHashJoin [codegen id : 22] +(75) BroadcastHashJoin [codegen id : 23] Left keys [1]: [ws_item_sk#59] Right keys [1]: [item_sk#22] Join condition: None -(76) Project [codegen id : 22] +(76) Project [codegen id : 23] Output [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] @@ -461,33 +461,33 @@ Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#64] -(78) Sort [codegen id : 23] +(78) Sort [codegen id : 24] Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 41] Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] -(80) HashAggregate [codegen id : 26] +(80) HashAggregate [codegen id : 27] Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] Keys [1]: [c_customer_sk#65] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] -(81) Filter [codegen id : 26] +(81) Filter [codegen id : 27] Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) -(82) Project [codegen id : 26] +(82) Project [codegen id : 27] Output [1]: [c_customer_sk#65] Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] -(83) Sort [codegen id : 26] +(83) Sort [codegen id : 27] Input [1]: [c_customer_sk#65] Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 -(84) SortMergeJoin +(84) SortMergeJoin [codegen id : 35] Left keys [1]: [ws_bill_customer_sk#60] Right keys [1]: [c_customer_sk#65] Join condition: None @@ -495,33 +495,33 @@ Join condition: None (85) ReusedExchange [Reuses operator id: 50] Output [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] -(86) Sort [codegen id : 28] +(86) Sort [codegen id : 29] Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] Arguments: [c_customer_sk#72 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 41] Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] -(88) HashAggregate [codegen id : 31] +(88) HashAggregate [codegen id : 32] Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] Keys [1]: [c_customer_sk#65] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] -(89) Filter [codegen id : 31] +(89) Filter [codegen id : 32] Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) -(90) Project [codegen id : 31] +(90) Project [codegen id : 32] Output [1]: [c_customer_sk#65] Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] -(91) Sort [codegen id : 31] +(91) Sort [codegen id : 32] Input [1]: [c_customer_sk#65] Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 -(92) SortMergeJoin +(92) SortMergeJoin [codegen id : 33] Left keys [1]: [c_customer_sk#72] Right keys [1]: [c_customer_sk#65] Join condition: None @@ -530,28 +530,28 @@ Join condition: None Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] -(94) BroadcastHashJoin [codegen id : 33] +(94) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ws_bill_customer_sk#60] Right keys [1]: [c_customer_sk#72] Join condition: None -(95) Project [codegen id : 33] +(95) Project [codegen id : 35] Output [5]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74] Input [7]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_customer_sk#72, c_first_name#73, c_last_name#74] (96) ReusedExchange [Reuses operator id: 65] Output [1]: [d_date_sk#76] -(97) BroadcastHashJoin [codegen id : 33] +(97) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ws_sold_date_sk#63] Right keys [1]: [d_date_sk#76] Join condition: None -(98) Project [codegen id : 33] +(98) Project [codegen id : 35] Output [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#76] -(99) HashAggregate [codegen id : 33] +(99) HashAggregate [codegen id : 35] Input [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] Keys [2]: [c_last_name#74, c_first_name#73] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] @@ -562,7 +562,7 @@ Results [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#81] -(101) HashAggregate [codegen id : 34] +(101) HashAggregate [codegen id : 36] Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] Keys [2]: [c_last_name#74, c_first_name#73] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index ac7e33fbc0f0c..8ef321be35bf0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -1,17 +1,17 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (16) + WholeStageCodegen (17) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter WholeStageCodegen (6) Sort [cs_bill_customer_sk] InputAdapter @@ -60,6 +60,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] + InputAdapter WholeStageCodegen (9) Sort [c_customer_sk] Project [c_customer_sk] @@ -122,49 +123,52 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Scan parquet default.customer [c_customer_sk] InputAdapter BroadcastExchange #14 - SortMergeJoin [c_customer_sk,c_customer_sk] - WholeStageCodegen (11) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #15 - WholeStageCodegen (10) - Filter [c_customer_sk] - ColumnarToRow + WholeStageCodegen (15) + SortMergeJoin [c_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (11) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #15 + WholeStageCodegen (10) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + WholeStageCodegen (14) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (14) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter BroadcastExchange #3 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (34) + WholeStageCodegen (36) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #16 - WholeStageCodegen (33) + WholeStageCodegen (35) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - WholeStageCodegen (23) + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (24) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #17 - WholeStageCodegen (22) + WholeStageCodegen (23) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] Filter [ws_bill_customer_sk] @@ -174,7 +178,8 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [item_sk] #18 - WholeStageCodegen (26) + InputAdapter + WholeStageCodegen (27) Sort [c_customer_sk] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -184,18 +189,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter BroadcastExchange #19 - SortMergeJoin [c_customer_sk,c_customer_sk] - WholeStageCodegen (28) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - WholeStageCodegen (31) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + WholeStageCodegen (33) + SortMergeJoin [c_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (29) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 + InputAdapter + WholeStageCodegen (32) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 9da5f36eb0553..9ed490449b008 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject (57) : : +- * Filter (34) : : +- SortMergeJoin ExistenceJoin(exists#1) (33) : : :- SortMergeJoin ExistenceJoin(exists#2) (25) - : : : :- SortMergeJoin LeftSemi (17) + : : : :- * SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) @@ -129,7 +129,7 @@ Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(17) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None @@ -141,18 +141,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 8] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#17] -(21) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#17] Join condition: None -(22) Project [codegen id : 7] +(22) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] @@ -160,7 +160,7 @@ Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] -(24) Sort [codegen id : 8] +(24) Sort [codegen id : 9] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 @@ -176,18 +176,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 11] Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#21] -(29) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#20] Right keys [1]: [d_date_sk#21] Join condition: None -(30) Project [codegen id : 10] +(30) Project [codegen id : 11] Output [1]: [cs_ship_customer_sk#19] Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] @@ -195,7 +195,7 @@ Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] Input [1]: [cs_ship_customer_sk#19] Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] -(32) Sort [codegen id : 11] +(32) Sort [codegen id : 12] Input [1]: [cs_ship_customer_sk#19] Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 @@ -204,11 +204,11 @@ Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#19] Join condition: None -(34) Filter [codegen id : 12] +(34) Filter [codegen id : 13] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(35) Project [codegen id : 12] +(35) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -216,7 +216,7 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] -(37) Sort [codegen id : 13] +(37) Sort [codegen id : 14] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 @@ -227,10 +227,10 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 14] +(39) ColumnarToRow [codegen id : 15] Input [2]: [ca_address_sk#24, ca_state#25] -(40) Filter [codegen id : 14] +(40) Filter [codegen id : 15] Input [2]: [ca_address_sk#24, ca_state#25] Condition : isnotnull(ca_address_sk#24) @@ -238,16 +238,16 @@ Condition : isnotnull(ca_address_sk#24) Input [2]: [ca_address_sk#24, ca_state#25] Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] -(42) Sort [codegen id : 15] +(42) Sort [codegen id : 16] Input [2]: [ca_address_sk#24, ca_state#25] Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 16] +(43) SortMergeJoin [codegen id : 17] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#24] Join condition: None -(44) Project [codegen id : 16] +(44) Project [codegen id : 17] Output [2]: [c_current_cdemo_sk#4, ca_state#25] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] @@ -255,7 +255,7 @@ Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_stat Input [2]: [c_current_cdemo_sk#4, ca_state#25] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] -(46) Sort [codegen id : 17] +(46) Sort [codegen id : 18] Input [2]: [c_current_cdemo_sk#4, ca_state#25] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 @@ -266,10 +266,10 @@ Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 18] +(48) ColumnarToRow [codegen id : 19] Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -(49) Filter [codegen id : 18] +(49) Filter [codegen id : 19] Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Condition : isnotnull(cd_demo_sk#28) @@ -277,20 +277,20 @@ Condition : isnotnull(cd_demo_sk#28) Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] -(51) Sort [codegen id : 19] +(51) Sort [codegen id : 20] Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 -(52) SortMergeJoin [codegen id : 20] +(52) SortMergeJoin [codegen id : 21] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#28] Join condition: None -(53) Project [codegen id : 20] +(53) Project [codegen id : 21] Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -(54) HashAggregate [codegen id : 20] +(54) HashAggregate [codegen id : 21] Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [10]: [partial_count(1), partial_min(cd_dep_count#31), partial_max(cd_dep_count#31), partial_avg(cd_dep_count#31), partial_min(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_avg(cd_dep_employed_count#32), partial_min(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_avg(cd_dep_college_count#33)] @@ -301,7 +301,7 @@ Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] -(56) HashAggregate [codegen id : 21] +(56) HashAggregate [codegen id : 22] Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [10]: [count(1), min(cd_dep_count#31), max(cd_dep_count#31), avg(cd_dep_count#31), min(cd_dep_employed_count#32), max(cd_dep_employed_count#32), avg(cd_dep_employed_count#32), min(cd_dep_college_count#33), max(cd_dep_college_count#33), avg(cd_dep_college_count#33)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt index 6c399f1bd2197..820be2b5fd58a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt @@ -1,66 +1,69 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (21) + WholeStageCodegen (22) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),aggOrder,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (20) + WholeStageCodegen (21) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (18) Sort [c_current_cdemo_sk] InputAdapter Exchange [c_current_cdemo_sk] #2 - WholeStageCodegen (16) + WholeStageCodegen (17) Project [c_current_cdemo_sk,ca_state] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (14) Sort [c_current_addr_sk] InputAdapter Exchange [c_current_addr_sk] #3 - WholeStageCodegen (12) + WholeStageCodegen (13) Project [c_current_cdemo_sk,c_current_addr_sk] Filter [exists,exists] InputAdapter SortMergeJoin [c_customer_sk,cs_ship_customer_sk] SortMergeJoin [c_customer_sk,ws_bill_customer_sk] - SortMergeJoin [c_customer_sk,ss_customer_sk] - WholeStageCodegen (2) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #4 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #5 - WholeStageCodegen (4) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - WholeStageCodegen (8) + WholeStageCodegen (6) + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (2) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #4 + WholeStageCodegen (1) + Filter [c_current_addr_sk,c_current_cdemo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #5 + WholeStageCodegen (4) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] + WholeStageCodegen (9) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #7 - WholeStageCodegen (7) + WholeStageCodegen (8) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -69,11 +72,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (11) + WholeStageCodegen (12) Sort [cs_ship_customer_sk] InputAdapter Exchange [cs_ship_customer_sk] #8 - WholeStageCodegen (10) + WholeStageCodegen (11) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -83,21 +86,21 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter - WholeStageCodegen (15) + WholeStageCodegen (16) Sort [ca_address_sk] InputAdapter Exchange [ca_address_sk] #9 - WholeStageCodegen (14) + WholeStageCodegen (15) Filter [ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - WholeStageCodegen (19) + WholeStageCodegen (20) Sort [cd_demo_sk] InputAdapter Exchange [cd_demo_sk] #10 - WholeStageCodegen (18) + WholeStageCodegen (19) Filter [cd_demo_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt index 71accc7295b0f..48edbc42d0ffb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt @@ -5,13 +5,13 @@ +- * HashAggregate (68) +- Exchange (67) +- * HashAggregate (66) - +- SortMergeJoin LeftSemi (65) + +- * SortMergeJoin LeftSemi (65) :- * Sort (47) : +- Exchange (46) : +- * HashAggregate (45) : +- Exchange (44) : +- * HashAggregate (43) - : +- SortMergeJoin LeftSemi (42) + : +- * SortMergeJoin LeftSemi (42) : :- * Sort (24) : : +- Exchange (23) : : +- * HashAggregate (22) @@ -260,7 +260,7 @@ Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), Input [3]: [c_last_name#22, c_first_name#21, d_date#18] Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 1970-01-01) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 -(42) SortMergeJoin +(42) SortMergeJoin [codegen id : 17] Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18)] Join condition: None @@ -368,7 +368,7 @@ Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), Input [3]: [c_last_name#34, c_first_name#33, d_date#30] Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 1970-01-01) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 -(65) SortMergeJoin +(65) SortMergeJoin [codegen id : 28] Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30)] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt index 2978f51532d83..421027136f3c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt @@ -9,8 +9,8 @@ WholeStageCodegen (30) Exchange [c_last_name,c_first_name,d_date] #2 WholeStageCodegen (28) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + InputAdapter WholeStageCodegen (19) Sort [c_last_name,c_first_name,d_date] InputAdapter @@ -21,8 +21,8 @@ WholeStageCodegen (30) Exchange [c_last_name,c_first_name,d_date] #4 WholeStageCodegen (17) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + InputAdapter WholeStageCodegen (8) Sort [c_last_name,c_first_name,d_date] InputAdapter @@ -67,6 +67,7 @@ WholeStageCodegen (30) ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter WholeStageCodegen (16) Sort [c_last_name,c_first_name,d_date] InputAdapter @@ -99,6 +100,7 @@ WholeStageCodegen (30) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #9 + InputAdapter WholeStageCodegen (27) Sort [c_last_name,c_first_name,d_date] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index a0532462eff64..2cb4a1f42eb1d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (51) : :- * Project (34) : : +- SortMergeJoin LeftAnti (33) : : :- SortMergeJoin LeftAnti (25) - : : : :- SortMergeJoin LeftSemi (17) + : : : :- * SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) @@ -123,7 +123,7 @@ Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(17) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None @@ -135,18 +135,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 8] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join condition: None -(22) Project [codegen id : 7] +(22) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#13] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] @@ -154,7 +154,7 @@ Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] Input [1]: [ws_bill_customer_sk#13] Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#16] -(24) Sort [codegen id : 8] +(24) Sort [codegen id : 9] Input [1]: [ws_bill_customer_sk#13] Arguments: [ws_bill_customer_sk#13 ASC NULLS FIRST], false, 0 @@ -170,18 +170,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 11] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#19] -(29) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#19] Join condition: None -(30) Project [codegen id : 10] +(30) Project [codegen id : 11] Output [1]: [cs_ship_customer_sk#17] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] @@ -189,7 +189,7 @@ Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] Input [1]: [cs_ship_customer_sk#17] Arguments: hashpartitioning(cs_ship_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#20] -(32) Sort [codegen id : 11] +(32) Sort [codegen id : 12] Input [1]: [cs_ship_customer_sk#17] Arguments: [cs_ship_customer_sk#17 ASC NULLS FIRST], false, 0 @@ -198,7 +198,7 @@ Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#17] Join condition: None -(34) Project [codegen id : 13] +(34) Project [codegen id : 14] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -209,14 +209,14 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 12] +(36) ColumnarToRow [codegen id : 13] Input [2]: [ca_address_sk#21, ca_state#22] -(37) Filter [codegen id : 12] +(37) Filter [codegen id : 13] Input [2]: [ca_address_sk#21, ca_state#22] Condition : (ca_state#22 IN (KY,GA,NM) AND isnotnull(ca_address_sk#21)) -(38) Project [codegen id : 12] +(38) Project [codegen id : 13] Output [1]: [ca_address_sk#21] Input [2]: [ca_address_sk#21, ca_state#22] @@ -224,12 +224,12 @@ Input [2]: [ca_address_sk#21, ca_state#22] Input [1]: [ca_address_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -(40) BroadcastHashJoin [codegen id : 13] +(40) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#21] Join condition: None -(41) Project [codegen id : 13] +(41) Project [codegen id : 14] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] @@ -251,16 +251,16 @@ Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_stat Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Condition : isnotnull(cd_demo_sk#25) -(46) BroadcastHashJoin [codegen id : 14] +(46) BroadcastHashJoin [codegen id : 15] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#25] Join condition: None -(47) Project [codegen id : 14] +(47) Project [codegen id : 15] Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] -(48) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 15] Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Functions [1]: [partial_count(1)] @@ -271,7 +271,7 @@ Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_pur Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, 5), ENSURE_REQUIREMENTS, [id=#33] -(50) HashAggregate [codegen id : 15] +(50) HashAggregate [codegen id : 16] Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt index dd7379392ec2f..bdbf95bd10721 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt @@ -1,56 +1,59 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (15) + WholeStageCodegen (16) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (14) + WholeStageCodegen (15) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (13) + WholeStageCodegen (14) Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_current_cdemo_sk,c_current_addr_sk] InputAdapter SortMergeJoin [c_customer_sk,cs_ship_customer_sk] SortMergeJoin [c_customer_sk,ws_bill_customer_sk] - SortMergeJoin [c_customer_sk,ss_customer_sk] - WholeStageCodegen (2) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #3 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (4) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (8) + WholeStageCodegen (6) + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (2) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #3 + WholeStageCodegen (1) + Filter [c_current_addr_sk,c_current_cdemo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #4 + WholeStageCodegen (4) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (9) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #6 - WholeStageCodegen (7) + WholeStageCodegen (8) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -59,11 +62,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) + WholeStageCodegen (12) Sort [cs_ship_customer_sk] InputAdapter Exchange [cs_ship_customer_sk] #7 - WholeStageCodegen (10) + WholeStageCodegen (11) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -74,7 +77,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #8 - WholeStageCodegen (12) + WholeStageCodegen (13) Project [ca_address_sk] Filter [ca_state,ca_address_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index 840f300eadff4..e06516011e4ec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -13,7 +13,7 @@ : : +- * BroadcastHashJoin Inner BuildRight (25) : : :- SortMergeJoin LeftAnti (19) : : : :- * Project (13) - : : : : +- SortMergeJoin LeftSemi (12) + : : : : +- * SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) @@ -94,7 +94,7 @@ Arguments: hashpartitioning(ws_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13 Input [2]: [ws_warehouse_sk#10, ws_order_number#11] Arguments: [ws_order_number#11 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin +(12) SortMergeJoin [codegen id : 5] Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#11] Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#10) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt index 4a60f03335730..b3c313fb5ded6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt @@ -17,8 +17,8 @@ WholeStageCodegen (12) SortMergeJoin [ws_order_number,wr_order_number] WholeStageCodegen (5) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - InputAdapter - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter WholeStageCodegen (2) Sort [ws_order_number] InputAdapter @@ -29,6 +29,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter WholeStageCodegen (4) Sort [ws_order_number] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 607f12f77e4c1..2d8c00cc4c936 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -13,7 +13,7 @@ : : +- * BroadcastHashJoin Inner BuildRight (25) : : :- SortMergeJoin LeftAnti (19) : : : :- * Project (13) - : : : : +- SortMergeJoin LeftSemi (12) + : : : : +- * SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) @@ -94,7 +94,7 @@ Arguments: hashpartitioning(ws_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13 Input [2]: [ws_warehouse_sk#10, ws_order_number#11] Arguments: [ws_order_number#11 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin +(12) SortMergeJoin [codegen id : 5] Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#11] Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#10) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 672601d740a87..cecad61df0774 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -17,8 +17,8 @@ WholeStageCodegen (12) SortMergeJoin [ws_order_number,wr_order_number] WholeStageCodegen (5) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - InputAdapter - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter WholeStageCodegen (2) Sort [ws_order_number] InputAdapter @@ -29,6 +29,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter WholeStageCodegen (4) Sort [ws_order_number] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index 82e4651d09edf..c9da298194fb8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -11,8 +11,8 @@ : +- * BroadcastHashJoin Inner BuildRight (44) : :- * Project (38) : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- SortMergeJoin LeftSemi (31) - : : : :- SortMergeJoin LeftSemi (17) + : : :- * SortMergeJoin LeftSemi (31) + : : : :- * SortMergeJoin LeftSemi (17) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) @@ -127,7 +127,7 @@ Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#13) Output [1]: [ws_order_number#10] Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] -(17) SortMergeJoin +(17) SortMergeJoin [codegen id : 8] Left keys [1]: [ws_order_number#4] Right keys [1]: [ws_order_number#10] Join condition: None @@ -139,14 +139,14 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 8] +(19) ColumnarToRow [codegen id : 9] Input [2]: [wr_order_number#15, wr_returned_date_sk#16] -(20) Filter [codegen id : 8] +(20) Filter [codegen id : 9] Input [2]: [wr_order_number#15, wr_returned_date_sk#16] Condition : isnotnull(wr_order_number#15) -(21) Project [codegen id : 8] +(21) Project [codegen id : 9] Output [1]: [wr_order_number#15] Input [2]: [wr_order_number#15, wr_returned_date_sk#16] @@ -154,18 +154,18 @@ Input [2]: [wr_order_number#15, wr_returned_date_sk#16] Input [1]: [wr_order_number#15] Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(23) Sort [codegen id : 9] +(23) Sort [codegen id : 10] Input [1]: [wr_order_number#15] Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (24) ReusedExchange [Reuses operator id: 11] Output [2]: [ws_warehouse_sk#18, ws_order_number#19] -(25) Sort [codegen id : 11] +(25) Sort [codegen id : 12] Input [2]: [ws_warehouse_sk#18, ws_order_number#19] Arguments: [ws_order_number#19 ASC NULLS FIRST], false, 0 -(26) SortMergeJoin [codegen id : 12] +(26) SortMergeJoin [codegen id : 13] Left keys [1]: [wr_order_number#15] Right keys [1]: [ws_order_number#19] Join condition: None @@ -173,20 +173,20 @@ Join condition: None (27) ReusedExchange [Reuses operator id: 11] Output [2]: [ws_warehouse_sk#20, ws_order_number#21] -(28) Sort [codegen id : 14] +(28) Sort [codegen id : 15] Input [2]: [ws_warehouse_sk#20, ws_order_number#21] Arguments: [ws_order_number#21 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 15] +(29) SortMergeJoin [codegen id : 16] Left keys [1]: [ws_order_number#19] Right keys [1]: [ws_order_number#21] Join condition: NOT (ws_warehouse_sk#18 = ws_warehouse_sk#20) -(30) Project [codegen id : 15] +(30) Project [codegen id : 16] Output [1]: [wr_order_number#15] Input [5]: [wr_order_number#15, ws_warehouse_sk#18, ws_order_number#19, ws_warehouse_sk#20, ws_order_number#21] -(31) SortMergeJoin +(31) SortMergeJoin [codegen id : 20] Left keys [1]: [ws_order_number#4] Right keys [1]: [wr_order_number#15] Join condition: None @@ -198,14 +198,14 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 16] +(33) ColumnarToRow [codegen id : 17] Input [2]: [ca_address_sk#22, ca_state#23] -(34) Filter [codegen id : 16] +(34) Filter [codegen id : 17] Input [2]: [ca_address_sk#22, ca_state#23] Condition : ((isnotnull(ca_state#23) AND (ca_state#23 = IL)) AND isnotnull(ca_address_sk#22)) -(35) Project [codegen id : 16] +(35) Project [codegen id : 17] Output [1]: [ca_address_sk#22] Input [2]: [ca_address_sk#22, ca_state#23] @@ -213,12 +213,12 @@ Input [2]: [ca_address_sk#22, ca_state#23] Input [1]: [ca_address_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(37) BroadcastHashJoin [codegen id : 19] +(37) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#22] Join condition: None -(38) Project [codegen id : 19] +(38) Project [codegen id : 20] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#22] @@ -229,14 +229,14 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 17] +(40) ColumnarToRow [codegen id : 18] Input [2]: [web_site_sk#25, web_company_name#26] -(41) Filter [codegen id : 17] +(41) Filter [codegen id : 18] Input [2]: [web_site_sk#25, web_company_name#26] Condition : ((isnotnull(web_company_name#26) AND (web_company_name#26 = pri )) AND isnotnull(web_site_sk#25)) -(42) Project [codegen id : 17] +(42) Project [codegen id : 18] Output [1]: [web_site_sk#25] Input [2]: [web_site_sk#25, web_company_name#26] @@ -244,12 +244,12 @@ Input [2]: [web_site_sk#25, web_company_name#26] Input [1]: [web_site_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] -(44) BroadcastHashJoin [codegen id : 19] +(44) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#25] Join condition: None -(45) Project [codegen id : 19] +(45) Project [codegen id : 20] Output [4]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#25] @@ -260,14 +260,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 18] +(47) ColumnarToRow [codegen id : 19] Input [2]: [d_date_sk#28, d_date#29] -(48) Filter [codegen id : 18] +(48) Filter [codegen id : 19] Input [2]: [d_date_sk#28, d_date#29] Condition : (((isnotnull(d_date#29) AND (d_date#29 >= 1999-02-01)) AND (d_date#29 <= 1999-04-02)) AND isnotnull(d_date_sk#28)) -(49) Project [codegen id : 18] +(49) Project [codegen id : 19] Output [1]: [d_date_sk#28] Input [2]: [d_date_sk#28, d_date#29] @@ -275,30 +275,30 @@ Input [2]: [d_date_sk#28, d_date#29] Input [1]: [d_date_sk#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] -(51) BroadcastHashJoin [codegen id : 19] +(51) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#28] Join condition: None -(52) Project [codegen id : 19] +(52) Project [codegen id : 20] Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#28] -(53) HashAggregate [codegen id : 19] +(53) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32] Results [3]: [ws_order_number#4, sum#33, sum#34] -(54) HashAggregate [codegen id : 19] +(54) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#33, sum#34] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32] Results [3]: [ws_order_number#4, sum#33, sum#34] -(55) HashAggregate [codegen id : 19] +(55) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#33, sum#34] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] @@ -309,14 +309,14 @@ Results [3]: [sum#33, sum#34, count#36] Input [3]: [sum#33, sum#34, count#36] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] -(57) HashAggregate [codegen id : 20] +(57) HashAggregate [codegen id : 21] Input [3]: [sum#33, sum#34, count#36] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32, count(ws_order_number#4)#35] Results [3]: [count(ws_order_number#4)#35 AS order count #38, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#31,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#32,17,2) AS total net profit #40] -(58) Sort [codegen id : 20] +(58) Sort [codegen id : 21] Input [3]: [order count #38, total shipping cost #39, total net profit #40] Arguments: [order count #38 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt index 83b9eed3cd2af..28d164854629b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (20) +WholeStageCodegen (21) Sort [order count ] HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (19) + WholeStageCodegen (20) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] @@ -13,68 +13,72 @@ WholeStageCodegen (20) BroadcastHashJoin [ws_web_site_sk,web_site_sk] Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - InputAdapter - SortMergeJoin [ws_order_number,wr_order_number] - SortMergeJoin [ws_order_number,ws_order_number] - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (8) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - WholeStageCodegen (7) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - Project [ws_warehouse_sk,ws_order_number] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - WholeStageCodegen (15) + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + Project [ws_warehouse_sk,ws_order_number] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (16) Project [wr_order_number] SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (13) SortMergeJoin [wr_order_number,ws_order_number] InputAdapter - WholeStageCodegen (9) + WholeStageCodegen (10) Sort [wr_order_number] InputAdapter Exchange [wr_order_number] #4 - WholeStageCodegen (8) + WholeStageCodegen (9) Project [wr_order_number] Filter [wr_order_number] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (12) Sort [ws_order_number] InputAdapter ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter - WholeStageCodegen (14) + WholeStageCodegen (15) Sort [ws_order_number] InputAdapter ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (16) + WholeStageCodegen (17) Project [ca_address_sk] Filter [ca_state,ca_address_sk] ColumnarToRow @@ -82,7 +86,7 @@ WholeStageCodegen (20) Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #6 - WholeStageCodegen (17) + WholeStageCodegen (18) Project [web_site_sk] Filter [web_company_name,web_site_sk] ColumnarToRow @@ -90,7 +94,7 @@ WholeStageCodegen (20) Scan parquet default.web_site [web_site_sk,web_company_name] InputAdapter BroadcastExchange #7 - WholeStageCodegen (18) + WholeStageCodegen (19) Project [d_date_sk] Filter [d_date,d_date_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 687e050b44090..253132c88e2c5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -11,8 +11,8 @@ : +- * BroadcastHashJoin Inner BuildRight (45) : :- * Project (39) : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- SortMergeJoin LeftSemi (32) - : : : :- SortMergeJoin LeftSemi (17) + : : :- * SortMergeJoin LeftSemi (32) + : : : :- * SortMergeJoin LeftSemi (17) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) @@ -128,7 +128,7 @@ Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#13) Output [1]: [ws_order_number#10] Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] -(17) SortMergeJoin +(17) SortMergeJoin [codegen id : 8] Left keys [1]: [ws_order_number#4] Right keys [1]: [ws_order_number#10] Join condition: None @@ -140,14 +140,14 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 8] +(19) ColumnarToRow [codegen id : 9] Input [2]: [wr_order_number#15, wr_returned_date_sk#16] -(20) Filter [codegen id : 8] +(20) Filter [codegen id : 9] Input [2]: [wr_order_number#15, wr_returned_date_sk#16] Condition : isnotnull(wr_order_number#15) -(21) Project [codegen id : 8] +(21) Project [codegen id : 9] Output [1]: [wr_order_number#15] Input [2]: [wr_order_number#15, wr_returned_date_sk#16] @@ -155,43 +155,43 @@ Input [2]: [wr_order_number#15, wr_returned_date_sk#16] Input [1]: [wr_order_number#15] Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(23) Sort [codegen id : 9] +(23) Sort [codegen id : 10] Input [1]: [wr_order_number#15] Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (24) ReusedExchange [Reuses operator id: 11] Output [2]: [ws_warehouse_sk#18, ws_order_number#19] -(25) Sort [codegen id : 11] +(25) Sort [codegen id : 12] Input [2]: [ws_warehouse_sk#18, ws_order_number#19] Arguments: [ws_order_number#19 ASC NULLS FIRST], false, 0 (26) ReusedExchange [Reuses operator id: 11] Output [2]: [ws_warehouse_sk#20, ws_order_number#21] -(27) Sort [codegen id : 13] +(27) Sort [codegen id : 14] Input [2]: [ws_warehouse_sk#20, ws_order_number#21] Arguments: [ws_order_number#21 ASC NULLS FIRST], false, 0 -(28) SortMergeJoin [codegen id : 14] +(28) SortMergeJoin [codegen id : 15] Left keys [1]: [ws_order_number#19] Right keys [1]: [ws_order_number#21] Join condition: NOT (ws_warehouse_sk#18 = ws_warehouse_sk#20) -(29) Project [codegen id : 14] +(29) Project [codegen id : 15] Output [1]: [ws_order_number#19] Input [4]: [ws_warehouse_sk#18, ws_order_number#19, ws_warehouse_sk#20, ws_order_number#21] -(30) SortMergeJoin [codegen id : 15] +(30) SortMergeJoin [codegen id : 16] Left keys [1]: [wr_order_number#15] Right keys [1]: [ws_order_number#19] Join condition: None -(31) Project [codegen id : 15] +(31) Project [codegen id : 16] Output [1]: [wr_order_number#15] Input [2]: [wr_order_number#15, ws_order_number#19] -(32) SortMergeJoin +(32) SortMergeJoin [codegen id : 20] Left keys [1]: [ws_order_number#4] Right keys [1]: [wr_order_number#15] Join condition: None @@ -203,14 +203,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 16] +(34) ColumnarToRow [codegen id : 17] Input [2]: [d_date_sk#22, d_date#23] -(35) Filter [codegen id : 16] +(35) Filter [codegen id : 17] Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1999-02-01)) AND (d_date#23 <= 1999-04-02)) AND isnotnull(d_date_sk#22)) -(36) Project [codegen id : 16] +(36) Project [codegen id : 17] Output [1]: [d_date_sk#22] Input [2]: [d_date_sk#22, d_date#23] @@ -218,12 +218,12 @@ Input [2]: [d_date_sk#22, d_date#23] Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(38) BroadcastHashJoin [codegen id : 19] +(38) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#22] Join condition: None -(39) Project [codegen id : 19] +(39) Project [codegen id : 20] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#22] @@ -234,14 +234,14 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 17] +(41) ColumnarToRow [codegen id : 18] Input [2]: [ca_address_sk#25, ca_state#26] -(42) Filter [codegen id : 17] +(42) Filter [codegen id : 18] Input [2]: [ca_address_sk#25, ca_state#26] Condition : ((isnotnull(ca_state#26) AND (ca_state#26 = IL)) AND isnotnull(ca_address_sk#25)) -(43) Project [codegen id : 17] +(43) Project [codegen id : 18] Output [1]: [ca_address_sk#25] Input [2]: [ca_address_sk#25, ca_state#26] @@ -249,12 +249,12 @@ Input [2]: [ca_address_sk#25, ca_state#26] Input [1]: [ca_address_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] -(45) BroadcastHashJoin [codegen id : 19] +(45) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#25] Join condition: None -(46) Project [codegen id : 19] +(46) Project [codegen id : 20] Output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#25] @@ -265,14 +265,14 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 18] +(48) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#28, web_company_name#29] -(49) Filter [codegen id : 18] +(49) Filter [codegen id : 19] Input [2]: [web_site_sk#28, web_company_name#29] Condition : ((isnotnull(web_company_name#29) AND (web_company_name#29 = pri )) AND isnotnull(web_site_sk#28)) -(50) Project [codegen id : 18] +(50) Project [codegen id : 19] Output [1]: [web_site_sk#28] Input [2]: [web_site_sk#28, web_company_name#29] @@ -280,30 +280,30 @@ Input [2]: [web_site_sk#28, web_company_name#29] Input [1]: [web_site_sk#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] -(52) BroadcastHashJoin [codegen id : 19] +(52) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#28] Join condition: None -(53) Project [codegen id : 19] +(53) Project [codegen id : 20] Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#28] -(54) HashAggregate [codegen id : 19] +(54) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32] Results [3]: [ws_order_number#4, sum#33, sum#34] -(55) HashAggregate [codegen id : 19] +(55) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#33, sum#34] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32] Results [3]: [ws_order_number#4, sum#33, sum#34] -(56) HashAggregate [codegen id : 19] +(56) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#33, sum#34] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] @@ -314,14 +314,14 @@ Results [3]: [sum#33, sum#34, count#36] Input [3]: [sum#33, sum#34, count#36] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] -(58) HashAggregate [codegen id : 20] +(58) HashAggregate [codegen id : 21] Input [3]: [sum#33, sum#34, count#36] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32, count(ws_order_number#4)#35] Results [3]: [count(ws_order_number#4)#35 AS order count #38, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#31,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#32,17,2) AS total net profit #40] -(59) Sort [codegen id : 20] +(59) Sort [codegen id : 21] Input [3]: [order count #38, total shipping cost #39, total net profit #40] Arguments: [order count #38 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 2372a40bd00d6..4d40e31545a21 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (20) +WholeStageCodegen (21) Sort [order count ] HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (19) + WholeStageCodegen (20) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] @@ -13,69 +13,73 @@ WholeStageCodegen (20) BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [ws_order_number,wr_order_number] - SortMergeJoin [ws_order_number,ws_order_number] - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (8) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - WholeStageCodegen (7) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - Project [ws_warehouse_sk,ws_order_number] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - WholeStageCodegen (15) + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + Project [ws_warehouse_sk,ws_order_number] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (16) Project [wr_order_number] SortMergeJoin [wr_order_number,ws_order_number] InputAdapter - WholeStageCodegen (9) + WholeStageCodegen (10) Sort [wr_order_number] InputAdapter Exchange [wr_order_number] #4 - WholeStageCodegen (8) + WholeStageCodegen (9) Project [wr_order_number] Filter [wr_order_number] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter - WholeStageCodegen (14) + WholeStageCodegen (15) Project [ws_order_number] SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (12) Sort [ws_order_number] InputAdapter ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (14) Sort [ws_order_number] InputAdapter ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (16) + WholeStageCodegen (17) Project [d_date_sk] Filter [d_date,d_date_sk] ColumnarToRow @@ -83,7 +87,7 @@ WholeStageCodegen (20) Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 - WholeStageCodegen (17) + WholeStageCodegen (18) Project [ca_address_sk] Filter [ca_state,ca_address_sk] ColumnarToRow @@ -91,7 +95,7 @@ WholeStageCodegen (20) Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (18) + WholeStageCodegen (19) Project [web_site_sk] Filter [web_company_name,web_site_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 2e710b7eda4b3..60b02bac254bb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -9,8 +9,8 @@ TakeOrderedAndProject (49) : +- * Project (39) : +- * BroadcastHashJoin Inner BuildRight (38) : :- * Project (32) - : : +- SortMergeJoin LeftSemi (31) - : : :- SortMergeJoin LeftSemi (17) + : : +- * SortMergeJoin LeftSemi (31) + : : :- * SortMergeJoin LeftSemi (17) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) @@ -121,7 +121,7 @@ Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(17) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None @@ -133,18 +133,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 8] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join condition: None -(22) Project [codegen id : 7] +(22) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#13 AS customer_sk#16] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] @@ -155,18 +155,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 9] +(24) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (25) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#19] -(26) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#19] Join condition: None -(27) Project [codegen id : 9] +(27) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#17 AS customer_sk#20] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] @@ -176,16 +176,16 @@ Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] Input [1]: [customer_sk#16] Arguments: hashpartitioning(customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#21] -(30) Sort [codegen id : 10] +(30) Sort [codegen id : 11] Input [1]: [customer_sk#16] Arguments: [customer_sk#16 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin +(31) SortMergeJoin [codegen id : 13] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#16] Join condition: None -(32) Project [codegen id : 12] +(32) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -196,14 +196,14 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 11] +(34) ColumnarToRow [codegen id : 12] Input [2]: [ca_address_sk#22, ca_county#23] -(35) Filter [codegen id : 11] +(35) Filter [codegen id : 12] Input [2]: [ca_address_sk#22, ca_county#23] Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) -(36) Project [codegen id : 11] +(36) Project [codegen id : 12] Output [1]: [ca_address_sk#22] Input [2]: [ca_address_sk#22, ca_county#23] @@ -211,12 +211,12 @@ Input [2]: [ca_address_sk#22, ca_county#23] Input [1]: [ca_address_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(38) BroadcastHashJoin [codegen id : 12] +(38) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#22] Join condition: None -(39) Project [codegen id : 12] +(39) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] @@ -238,16 +238,16 @@ Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_stat Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Condition : isnotnull(cd_demo_sk#26) -(44) BroadcastHashJoin [codegen id : 13] +(44) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#26] Join condition: None -(45) Project [codegen id : 13] +(45) Project [codegen id : 14] Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -(46) HashAggregate [codegen id : 13] +(46) HashAggregate [codegen id : 14] Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Functions [1]: [partial_count(1)] @@ -258,7 +258,7 @@ Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_pur Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, 5), ENSURE_REQUIREMENTS, [id=#37] -(48) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 15] Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt index 220059a6f5e27..433cb374a4c75 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt @@ -1,56 +1,60 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (14) + WholeStageCodegen (15) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (13) + WholeStageCodegen (14) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (12) + WholeStageCodegen (13) Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - SortMergeJoin [c_customer_sk,customer_sk] - SortMergeJoin [c_customer_sk,ss_customer_sk] - WholeStageCodegen (2) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #3 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (4) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (10) + SortMergeJoin [c_customer_sk,customer_sk] + InputAdapter + WholeStageCodegen (6) + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (2) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #3 + WholeStageCodegen (1) + Filter [c_current_addr_sk,c_current_cdemo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #4 + WholeStageCodegen (4) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + WholeStageCodegen (11) Sort [customer_sk] InputAdapter Exchange [customer_sk] #6 Union - WholeStageCodegen (7) + WholeStageCodegen (8) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -59,7 +63,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 - WholeStageCodegen (9) + WholeStageCodegen (10) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -70,7 +74,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #7 - WholeStageCodegen (11) + WholeStageCodegen (12) Project [ca_address_sk] Filter [ca_county,ca_address_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index 25baf22f2b4b5..e1ce480288a45 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (116) : +- * BroadcastHashJoin Inner BuildRight (84) : :- * Project (74) : : +- * BroadcastHashJoin Inner BuildRight (73) - : : :- SortMergeJoin LeftSemi (67) + : : :- * SortMergeJoin LeftSemi (67) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) @@ -27,7 +27,7 @@ TakeOrderedAndProject (116) : : : +- * HashAggregate (61) : : : +- Exchange (60) : : : +- * HashAggregate (59) - : : : +- SortMergeJoin LeftSemi (58) + : : : +- * SortMergeJoin LeftSemi (58) : : : :- * Sort (46) : : : : +- Exchange (45) : : : : +- * HashAggregate (44) @@ -46,7 +46,7 @@ TakeOrderedAndProject (116) : : : : : +- * ColumnarToRow (13) : : : : : +- Scan parquet default.date_dim (12) : : : : +- BroadcastExchange (39) - : : : : +- SortMergeJoin LeftSemi (38) + : : : : +- * SortMergeJoin LeftSemi (38) : : : : :- * Sort (23) : : : : : +- Exchange (22) : : : : : +- * Filter (21) @@ -83,7 +83,7 @@ TakeOrderedAndProject (116) : : +- * ColumnarToRow (69) : : +- Scan parquet default.date_dim (68) : +- BroadcastExchange (83) - : +- SortMergeJoin LeftSemi (82) + : +- * SortMergeJoin LeftSemi (82) : :- * Sort (79) : : +- Exchange (78) : : +- * Filter (77) @@ -101,7 +101,7 @@ TakeOrderedAndProject (116) +- * BroadcastHashJoin Inner BuildRight (107) :- * Project (105) : +- * BroadcastHashJoin Inner BuildRight (104) - : :- SortMergeJoin LeftSemi (98) + : :- * SortMergeJoin LeftSemi (98) : : :- * Sort (95) : : : +- Exchange (94) : : : +- * Filter (93) @@ -147,10 +147,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -162,10 +162,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] +(10) ColumnarToRow [codegen id : 11] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 10] +(11) Filter [codegen id : 11] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Condition : isnotnull(ss_item_sk#11) @@ -191,12 +191,12 @@ Input [2]: [d_date_sk#14, d_year#15] Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 10] +(17) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#12] Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 10] +(18) Project [codegen id : 11] Output [1]: [ss_item_sk#11] Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] @@ -284,7 +284,7 @@ Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), c Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(38) SortMergeJoin +(38) SortMergeJoin [codegen id : 10] Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None @@ -293,16 +293,16 @@ Join condition: None Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(40) BroadcastHashJoin [codegen id : 10] +(40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#11] Right keys [1]: [i_item_sk#17] Join condition: None -(41) Project [codegen id : 10] +(41) Project [codegen id : 11] Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 11] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -313,7 +313,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 12] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -324,7 +324,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 12] +(46) Sort [codegen id : 13] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 @@ -336,34 +336,34 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] +(48) ColumnarToRow [codegen id : 16] Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 15] +(49) Filter [codegen id : 16] Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 15] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_sold_date_sk#38] Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 15] +(52) Project [codegen id : 16] Output [1]: [ws_item_sk#37] Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] (53) ReusedExchange [Reuses operator id: 33] Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 15] +(54) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_item_sk#37] Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 15] +(55) Project [codegen id : 16] Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] @@ -371,16 +371,16 @@ Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_categor Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 16] +(57) Sort [codegen id : 17] Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 -(58) SortMergeJoin +(58) SortMergeJoin [codegen id : 18] Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 17] +(59) HashAggregate [codegen id : 18] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -391,7 +391,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 18] +(61) HashAggregate [codegen id : 19] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -402,12 +402,12 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 19] +(64) Project [codegen id : 20] Output [1]: [i_item_sk#7 AS ss_item_sk#47] Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] @@ -415,11 +415,11 @@ Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id# Input [1]: [ss_item_sk#47] Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 20] +(66) Sort [codegen id : 21] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(67) SortMergeJoin +(67) SortMergeJoin [codegen id : 45] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -431,14 +431,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 21] +(69) ColumnarToRow [codegen id : 22] Input [2]: [d_date_sk#49, d_week_seq#50] -(70) Filter [codegen id : 21] +(70) Filter [codegen id : 22] Input [2]: [d_date_sk#49, d_week_seq#50] Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 21] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#49] Input [2]: [d_date_sk#49, d_week_seq#50] @@ -446,12 +446,12 @@ Input [2]: [d_date_sk#49, d_week_seq#50] Input [1]: [d_date_sk#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 45] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 45] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] @@ -462,10 +462,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(77) Filter [codegen id : 22] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) @@ -473,18 +473,18 @@ Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnu Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] -(79) Sort [codegen id : 23] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 41] +(81) Sort [codegen id : 43] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(82) SortMergeJoin +(82) SortMergeJoin [codegen id : 44] Left keys [1]: [i_item_sk#54] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -493,16 +493,16 @@ Join condition: None Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 45] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#54] Join condition: None -(85) Project [codegen id : 42] +(85) Project [codegen id : 45] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(86) HashAggregate [codegen id : 42] +(86) HashAggregate [codegen id : 45] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] @@ -513,18 +513,18 @@ Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64 Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] -(88) HashAggregate [codegen id : 86] +(88) HashAggregate [codegen id : 92] Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -(89) Filter [codegen id : 86] +(89) Filter [codegen id : 92] Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(90) Project [codegen id : 86] +(90) Project [codegen id : 92] Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] @@ -536,10 +536,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 43] +(92) ColumnarToRow [codegen id : 46] Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -(93) Filter [codegen id : 43] +(93) Filter [codegen id : 46] Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Condition : isnotnull(ss_item_sk#75) @@ -547,18 +547,18 @@ Condition : isnotnull(ss_item_sk#75) Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] -(95) Sort [codegen id : 44] +(95) Sort [codegen id : 47] Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 62] +(97) Sort [codegen id : 66] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin +(98) SortMergeJoin [codegen id : 90] Left keys [1]: [ss_item_sk#75] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -570,14 +570,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 63] +(100) ColumnarToRow [codegen id : 67] Input [2]: [d_date_sk#81, d_week_seq#82] -(101) Filter [codegen id : 63] +(101) Filter [codegen id : 67] Input [2]: [d_date_sk#81, d_week_seq#82] Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) -(102) Project [codegen id : 63] +(102) Project [codegen id : 67] Output [1]: [d_date_sk#81] Input [2]: [d_date_sk#81, d_week_seq#82] @@ -585,28 +585,28 @@ Input [2]: [d_date_sk#81, d_week_seq#82] Input [1]: [d_date_sk#81] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] -(104) BroadcastHashJoin [codegen id : 84] +(104) BroadcastHashJoin [codegen id : 90] Left keys [1]: [ss_sold_date_sk#78] Right keys [1]: [d_date_sk#81] Join condition: None -(105) Project [codegen id : 84] +(105) Project [codegen id : 90] Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] (106) ReusedExchange [Reuses operator id: 83] Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(107) BroadcastHashJoin [codegen id : 84] +(107) BroadcastHashJoin [codegen id : 90] Left keys [1]: [ss_item_sk#75] Right keys [1]: [i_item_sk#86] Join condition: None -(108) Project [codegen id : 84] +(108) Project [codegen id : 90] Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(109) HashAggregate [codegen id : 84] +(109) HashAggregate [codegen id : 90] Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] @@ -617,18 +617,18 @@ Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94 Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] -(111) HashAggregate [codegen id : 85] +(111) HashAggregate [codegen id : 91] Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -(112) Filter [codegen id : 85] +(112) Filter [codegen id : 91] Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(113) Project [codegen id : 85] +(113) Project [codegen id : 91] Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] @@ -636,7 +636,7 @@ Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sal Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] -(115) BroadcastHashJoin [codegen id : 86] +(115) BroadcastHashJoin [codegen id : 92] Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index 1b41abbc97b06..6a8fcada7a61b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (86) + WholeStageCodegen (92) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,14 +49,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (42) + WholeStageCodegen (45) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk,ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [ss_item_sk,ss_item_sk] + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter WholeStageCodegen (2) Sort [ss_item_sk] InputAdapter @@ -68,11 +68,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (20) + InputAdapter + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,23 +82,23 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -119,44 +120,48 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #9 InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -172,7 +177,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -187,55 +192,59 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #15 - SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #16 - WholeStageCodegen (22) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk] #17 + WholeStageCodegen (44) + SortMergeJoin [i_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (24) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #16 + WholeStageCodegen (23) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (43) + Sort [ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (85) + WholeStageCodegen (91) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (84) + WholeStageCodegen (90) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk,ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (44) + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (47) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (43) + WholeStageCodegen (46) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (62) + InputAdapter + WholeStageCodegen (66) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (63) + WholeStageCodegen (67) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt index 43ebe6e0b4d9b..ef5f915e1b17e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject (213) : : +- * BroadcastHashJoin Inner BuildRight (84) : : :- * Project (74) : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- SortMergeJoin LeftSemi (67) + : : : :- * SortMergeJoin LeftSemi (67) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) @@ -34,7 +34,7 @@ TakeOrderedAndProject (213) : : : : +- * HashAggregate (61) : : : : +- Exchange (60) : : : : +- * HashAggregate (59) - : : : : +- SortMergeJoin LeftSemi (58) + : : : : +- * SortMergeJoin LeftSemi (58) : : : : :- * Sort (46) : : : : : +- Exchange (45) : : : : : +- * HashAggregate (44) @@ -53,7 +53,7 @@ TakeOrderedAndProject (213) : : : : : : +- * ColumnarToRow (13) : : : : : : +- Scan parquet default.date_dim (12) : : : : : +- BroadcastExchange (39) - : : : : : +- SortMergeJoin LeftSemi (38) + : : : : : +- * SortMergeJoin LeftSemi (38) : : : : : :- * Sort (23) : : : : : : +- Exchange (22) : : : : : : +- * Filter (21) @@ -90,7 +90,7 @@ TakeOrderedAndProject (213) : : : +- * ColumnarToRow (69) : : : +- Scan parquet default.date_dim (68) : : +- BroadcastExchange (83) - : : +- SortMergeJoin LeftSemi (82) + : : +- * SortMergeJoin LeftSemi (82) : : :- * Sort (79) : : : +- Exchange (78) : : : +- * Filter (77) @@ -107,7 +107,7 @@ TakeOrderedAndProject (213) : : +- * BroadcastHashJoin Inner BuildRight (103) : : :- * Project (101) : : : +- * BroadcastHashJoin Inner BuildRight (100) - : : : :- SortMergeJoin LeftSemi (98) + : : : :- * SortMergeJoin LeftSemi (98) : : : : :- * Sort (95) : : : : : +- Exchange (94) : : : : : +- * Filter (93) @@ -126,7 +126,7 @@ TakeOrderedAndProject (213) : +- * BroadcastHashJoin Inner BuildRight (122) : :- * Project (120) : : +- * BroadcastHashJoin Inner BuildRight (119) - : : :- SortMergeJoin LeftSemi (117) + : : :- * SortMergeJoin LeftSemi (117) : : : :- * Sort (114) : : : : +- Exchange (113) : : : : +- * Filter (112) @@ -244,10 +244,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -259,10 +259,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] +(10) ColumnarToRow [codegen id : 11] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 10] +(11) Filter [codegen id : 11] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Condition : isnotnull(ss_item_sk#11) @@ -288,12 +288,12 @@ Input [2]: [d_date_sk#14, d_year#15] Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 10] +(17) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#12] Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 10] +(18) Project [codegen id : 11] Output [1]: [ss_item_sk#11] Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] @@ -381,7 +381,7 @@ Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), c Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(38) SortMergeJoin +(38) SortMergeJoin [codegen id : 10] Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None @@ -390,16 +390,16 @@ Join condition: None Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(40) BroadcastHashJoin [codegen id : 10] +(40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#11] Right keys [1]: [i_item_sk#17] Join condition: None -(41) Project [codegen id : 10] +(41) Project [codegen id : 11] Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 11] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -410,7 +410,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 12] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -421,7 +421,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 12] +(46) Sort [codegen id : 13] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 @@ -433,34 +433,34 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] +(48) ColumnarToRow [codegen id : 16] Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 15] +(49) Filter [codegen id : 16] Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 15] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_sold_date_sk#38] Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 15] +(52) Project [codegen id : 16] Output [1]: [ws_item_sk#37] Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] (53) ReusedExchange [Reuses operator id: 33] Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 15] +(54) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_item_sk#37] Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 15] +(55) Project [codegen id : 16] Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] @@ -468,16 +468,16 @@ Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_categor Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 16] +(57) Sort [codegen id : 17] Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 -(58) SortMergeJoin +(58) SortMergeJoin [codegen id : 18] Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 17] +(59) HashAggregate [codegen id : 18] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -488,7 +488,7 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 18] +(61) HashAggregate [codegen id : 19] Input [3]: [brand_id#32, class_id#33, category_id#34] Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] @@ -499,12 +499,12 @@ Results [3]: [brand_id#32, class_id#33, category_id#34] Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 19] +(64) Project [codegen id : 20] Output [1]: [i_item_sk#7 AS ss_item_sk#47] Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] @@ -512,11 +512,11 @@ Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id# Input [1]: [ss_item_sk#47] Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 20] +(66) Sort [codegen id : 21] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(67) SortMergeJoin +(67) SortMergeJoin [codegen id : 45] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -528,14 +528,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 21] +(69) ColumnarToRow [codegen id : 22] Input [3]: [d_date_sk#49, d_year#50, d_moy#51] -(70) Filter [codegen id : 21] +(70) Filter [codegen id : 22] Input [3]: [d_date_sk#49, d_year#50, d_moy#51] Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2000)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 21] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#49] Input [3]: [d_date_sk#49, d_year#50, d_moy#51] @@ -543,12 +543,12 @@ Input [3]: [d_date_sk#49, d_year#50, d_moy#51] Input [1]: [d_date_sk#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 45] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 45] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] @@ -559,10 +559,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(77) Filter [codegen id : 22] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Condition : isnotnull(i_item_sk#53) @@ -570,18 +570,18 @@ Condition : isnotnull(i_item_sk#53) Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] -(79) Sort [codegen id : 23] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 41] +(81) Sort [codegen id : 43] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(82) SortMergeJoin +(82) SortMergeJoin [codegen id : 44] Left keys [1]: [i_item_sk#53] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -590,16 +590,16 @@ Join condition: None Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 45] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#53] Join condition: None -(85) Project [codegen id : 42] +(85) Project [codegen id : 45] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(86) HashAggregate [codegen id : 42] +(86) HashAggregate [codegen id : 45] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] @@ -610,18 +610,18 @@ Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63 Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] -(88) HashAggregate [codegen id : 43] +(88) HashAggregate [codegen id : 46] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -(89) Filter [codegen id : 43] +(89) Filter [codegen id : 46] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(90) Project [codegen id : 43] +(90) Project [codegen id : 46] Output [6]: [store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] @@ -633,10 +633,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 44] +(92) ColumnarToRow [codegen id : 47] Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -(93) Filter [codegen id : 44] +(93) Filter [codegen id : 47] Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Condition : isnotnull(cs_item_sk#74) @@ -644,18 +644,18 @@ Condition : isnotnull(cs_item_sk#74) Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] -(95) Sort [codegen id : 45] +(95) Sort [codegen id : 48] Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 63] +(97) Sort [codegen id : 67] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin +(98) SortMergeJoin [codegen id : 91] Left keys [1]: [cs_item_sk#74] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -663,28 +663,28 @@ Join condition: None (99) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#79] -(100) BroadcastHashJoin [codegen id : 85] +(100) BroadcastHashJoin [codegen id : 91] Left keys [1]: [cs_sold_date_sk#77] Right keys [1]: [d_date_sk#79] Join condition: None -(101) Project [codegen id : 85] +(101) Project [codegen id : 91] Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] (102) ReusedExchange [Reuses operator id: 83] Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(103) BroadcastHashJoin [codegen id : 85] +(103) BroadcastHashJoin [codegen id : 91] Left keys [1]: [cs_item_sk#74] Right keys [1]: [i_item_sk#80] Join condition: None -(104) Project [codegen id : 85] +(104) Project [codegen id : 91] Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(105) HashAggregate [codegen id : 85] +(105) HashAggregate [codegen id : 91] Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] @@ -695,18 +695,18 @@ Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88 Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] -(107) HashAggregate [codegen id : 86] +(107) HashAggregate [codegen id : 92] Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -(108) Filter [codegen id : 86] +(108) Filter [codegen id : 92] Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(109) Project [codegen id : 86] +(109) Project [codegen id : 92] Output [6]: [catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94] Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] @@ -718,10 +718,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 87] +(111) ColumnarToRow [codegen id : 93] Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -(112) Filter [codegen id : 87] +(112) Filter [codegen id : 93] Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Condition : isnotnull(ws_item_sk#97) @@ -729,18 +729,18 @@ Condition : isnotnull(ws_item_sk#97) Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] -(114) Sort [codegen id : 88] +(114) Sort [codegen id : 94] Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(116) Sort [codegen id : 106] +(116) Sort [codegen id : 113] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(117) SortMergeJoin +(117) SortMergeJoin [codegen id : 137] Left keys [1]: [ws_item_sk#97] Right keys [1]: [ss_item_sk#47] Join condition: None @@ -748,28 +748,28 @@ Join condition: None (118) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#102] -(119) BroadcastHashJoin [codegen id : 128] +(119) BroadcastHashJoin [codegen id : 137] Left keys [1]: [ws_sold_date_sk#100] Right keys [1]: [d_date_sk#102] Join condition: None -(120) Project [codegen id : 128] +(120) Project [codegen id : 137] Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] (121) ReusedExchange [Reuses operator id: 83] Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(122) BroadcastHashJoin [codegen id : 128] +(122) BroadcastHashJoin [codegen id : 137] Left keys [1]: [ws_item_sk#97] Right keys [1]: [i_item_sk#103] Join condition: None -(123) Project [codegen id : 128] +(123) Project [codegen id : 137] Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(124) HashAggregate [codegen id : 128] +(124) HashAggregate [codegen id : 137] Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] @@ -780,24 +780,24 @@ Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpt Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] -(126) HashAggregate [codegen id : 129] +(126) HashAggregate [codegen id : 138] Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -(127) Filter [codegen id : 129] +(127) Filter [codegen id : 138] Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(128) Project [codegen id : 129] +(128) Project [codegen id : 138] Output [6]: [web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117] Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] (129) Union -(130) HashAggregate [codegen id : 130] +(130) HashAggregate [codegen id : 139] Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] @@ -808,7 +808,7 @@ Results [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#12 Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#126] -(132) HashAggregate [codegen id : 131] +(132) HashAggregate [codegen id : 140] Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(sales#68), sum(number_sales#69)] @@ -818,60 +818,60 @@ Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum(sa (133) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] -(134) HashAggregate [codegen id : 174] +(134) HashAggregate [codegen id : 186] Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] Keys [3]: [i_brand_id#131, i_class_id#132, i_category_id#133] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] Results [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#68, count(1)#140 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] -(135) Filter [codegen id : 174] +(135) Filter [codegen id : 186] Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(136) Project [codegen id : 174] +(136) Project [codegen id : 186] Output [6]: [store AS channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] (137) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] -(138) HashAggregate [codegen id : 217] +(138) HashAggregate [codegen id : 232] Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] Keys [3]: [i_brand_id#142, i_class_id#143, i_category_id#144] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150, count(1)#151] Results [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sales#93, count(1)#151 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] -(139) Filter [codegen id : 217] +(139) Filter [codegen id : 232] Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(140) Project [codegen id : 217] +(140) Project [codegen id : 232] Output [6]: [catalog AS channel#96, i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94] Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] (141) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] -(142) HashAggregate [codegen id : 260] +(142) HashAggregate [codegen id : 278] Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] Keys [3]: [i_brand_id#153, i_class_id#154, i_category_id#155] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161, count(1)#162] Results [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sales#116, count(1)#162 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] -(143) Filter [codegen id : 260] +(143) Filter [codegen id : 278] Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(144) Project [codegen id : 260] +(144) Project [codegen id : 278] Output [6]: [web AS channel#119, i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117] Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] (145) Union -(146) HashAggregate [codegen id : 261] +(146) HashAggregate [codegen id : 279] Input [6]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] @@ -882,14 +882,14 @@ Results [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, 5), ENSURE_REQUIREMENTS, [id=#170] -(148) HashAggregate [codegen id : 262] +(148) HashAggregate [codegen id : 280] Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] Functions [2]: [sum(sales#68), sum(number_sales#69)] Aggregate Attributes [2]: [sum(sales#68)#171, sum(number_sales#69)#172] Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#171 AS sum_sales#129, sum(number_sales#69)#172 AS number_sales#130] -(149) HashAggregate [codegen id : 262] +(149) HashAggregate [codegen id : 280] Input [5]: [channel#73, i_brand_id#131, i_class_id#132, sum_sales#129, number_sales#130] Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] @@ -900,7 +900,7 @@ Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#179] -(151) HashAggregate [codegen id : 263] +(151) HashAggregate [codegen id : 281] Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] @@ -910,60 +910,60 @@ Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id# (152) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] -(153) HashAggregate [codegen id : 306] +(153) HashAggregate [codegen id : 327] Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] Keys [3]: [i_brand_id#185, i_class_id#186, i_category_id#187] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193, count(1)#194] Results [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sales#68, count(1)#194 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] -(154) Filter [codegen id : 306] +(154) Filter [codegen id : 327] Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(155) Project [codegen id : 306] +(155) Project [codegen id : 327] Output [6]: [store AS channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] (156) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] -(157) HashAggregate [codegen id : 349] +(157) HashAggregate [codegen id : 373] Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] Keys [3]: [i_brand_id#196, i_class_id#197, i_category_id#198] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204, count(1)#205] Results [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sales#93, count(1)#205 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] -(158) Filter [codegen id : 349] +(158) Filter [codegen id : 373] Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(159) Project [codegen id : 349] +(159) Project [codegen id : 373] Output [6]: [catalog AS channel#96, i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94] Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] (160) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] -(161) HashAggregate [codegen id : 392] +(161) HashAggregate [codegen id : 419] Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] Keys [3]: [i_brand_id#207, i_class_id#208, i_category_id#209] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215, count(1)#216] Results [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sales#116, count(1)#216 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] -(162) Filter [codegen id : 392] +(162) Filter [codegen id : 419] Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(163) Project [codegen id : 392] +(163) Project [codegen id : 419] Output [6]: [web AS channel#119, i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117] Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] (164) Union -(165) HashAggregate [codegen id : 393] +(165) HashAggregate [codegen id : 420] Input [6]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] @@ -974,14 +974,14 @@ Results [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] Arguments: hashpartitioning(channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, 5), ENSURE_REQUIREMENTS, [id=#224] -(167) HashAggregate [codegen id : 394] +(167) HashAggregate [codegen id : 421] Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] Functions [2]: [sum(sales#68), sum(number_sales#69)] Aggregate Attributes [2]: [sum(sales#68)#225, sum(number_sales#69)#226] Results [4]: [channel#73, i_brand_id#185, sum(sales#68)#225 AS sum_sales#129, sum(number_sales#69)#226 AS number_sales#130] -(168) HashAggregate [codegen id : 394] +(168) HashAggregate [codegen id : 421] Input [4]: [channel#73, i_brand_id#185, sum_sales#129, number_sales#130] Keys [2]: [channel#73, i_brand_id#185] Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] @@ -992,7 +992,7 @@ Results [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] Arguments: hashpartitioning(channel#73, i_brand_id#185, 5), ENSURE_REQUIREMENTS, [id=#233] -(170) HashAggregate [codegen id : 395] +(170) HashAggregate [codegen id : 422] Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] Keys [2]: [channel#73, i_brand_id#185] Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] @@ -1002,60 +1002,60 @@ Results [6]: [channel#73, i_brand_id#185, null AS i_class_id#236, null AS i_cate (171) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] -(172) HashAggregate [codegen id : 438] +(172) HashAggregate [codegen id : 468] Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] Keys [3]: [i_brand_id#240, i_class_id#241, i_category_id#242] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248, count(1)#249] Results [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sales#68, count(1)#249 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] -(173) Filter [codegen id : 438] +(173) Filter [codegen id : 468] Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(174) Project [codegen id : 438] +(174) Project [codegen id : 468] Output [6]: [store AS channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] (175) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] -(176) HashAggregate [codegen id : 481] +(176) HashAggregate [codegen id : 514] Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] Keys [3]: [i_brand_id#251, i_class_id#252, i_category_id#253] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259, count(1)#260] Results [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sales#93, count(1)#260 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] -(177) Filter [codegen id : 481] +(177) Filter [codegen id : 514] Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(178) Project [codegen id : 481] +(178) Project [codegen id : 514] Output [6]: [catalog AS channel#96, i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94] Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] (179) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] -(180) HashAggregate [codegen id : 524] +(180) HashAggregate [codegen id : 560] Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] Keys [3]: [i_brand_id#262, i_class_id#263, i_category_id#264] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270, count(1)#271] Results [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sales#116, count(1)#271 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] -(181) Filter [codegen id : 524] +(181) Filter [codegen id : 560] Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(182) Project [codegen id : 524] +(182) Project [codegen id : 560] Output [6]: [web AS channel#119, i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117] Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] (183) Union -(184) HashAggregate [codegen id : 525] +(184) HashAggregate [codegen id : 561] Input [6]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] @@ -1066,14 +1066,14 @@ Results [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] Arguments: hashpartitioning(channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, 5), ENSURE_REQUIREMENTS, [id=#279] -(186) HashAggregate [codegen id : 526] +(186) HashAggregate [codegen id : 562] Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] Functions [2]: [sum(sales#68), sum(number_sales#69)] Aggregate Attributes [2]: [sum(sales#68)#280, sum(number_sales#69)#281] Results [3]: [channel#73, sum(sales#68)#280 AS sum_sales#129, sum(number_sales#69)#281 AS number_sales#130] -(187) HashAggregate [codegen id : 526] +(187) HashAggregate [codegen id : 562] Input [3]: [channel#73, sum_sales#129, number_sales#130] Keys [1]: [channel#73] Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] @@ -1084,7 +1084,7 @@ Results [4]: [channel#73, sum#285, isEmpty#286, sum#287] Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#288] -(189) HashAggregate [codegen id : 527] +(189) HashAggregate [codegen id : 563] Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] Keys [1]: [channel#73] Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] @@ -1094,60 +1094,60 @@ Results [6]: [channel#73, null AS i_brand_id#291, null AS i_class_id#292, null A (190) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] -(191) HashAggregate [codegen id : 570] +(191) HashAggregate [codegen id : 609] Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] Keys [3]: [i_brand_id#296, i_class_id#297, i_category_id#298] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304, count(1)#305] Results [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sales#68, count(1)#305 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] -(192) Filter [codegen id : 570] +(192) Filter [codegen id : 609] Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(193) Project [codegen id : 570] +(193) Project [codegen id : 609] Output [6]: [store AS channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] (194) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] -(195) HashAggregate [codegen id : 613] +(195) HashAggregate [codegen id : 655] Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] Keys [3]: [i_brand_id#307, i_class_id#308, i_category_id#309] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315, count(1)#316] Results [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sales#93, count(1)#316 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] -(196) Filter [codegen id : 613] +(196) Filter [codegen id : 655] Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(197) Project [codegen id : 613] +(197) Project [codegen id : 655] Output [6]: [catalog AS channel#96, i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94] Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] (198) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] -(199) HashAggregate [codegen id : 656] +(199) HashAggregate [codegen id : 701] Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] Keys [3]: [i_brand_id#318, i_class_id#319, i_category_id#320] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326, count(1)#327] Results [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sales#116, count(1)#327 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] -(200) Filter [codegen id : 656] +(200) Filter [codegen id : 701] Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(201) Project [codegen id : 656] +(201) Project [codegen id : 701] Output [6]: [web AS channel#119, i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117] Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] (202) Union -(203) HashAggregate [codegen id : 657] +(203) HashAggregate [codegen id : 702] Input [6]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] @@ -1158,14 +1158,14 @@ Results [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] Arguments: hashpartitioning(channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, 5), ENSURE_REQUIREMENTS, [id=#335] -(205) HashAggregate [codegen id : 658] +(205) HashAggregate [codegen id : 703] Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] Functions [2]: [sum(sales#68), sum(number_sales#69)] Aggregate Attributes [2]: [sum(sales#68)#336, sum(number_sales#69)#337] Results [2]: [sum(sales#68)#336 AS sum_sales#129, sum(number_sales#69)#337 AS number_sales#130] -(206) HashAggregate [codegen id : 658] +(206) HashAggregate [codegen id : 703] Input [2]: [sum_sales#129, number_sales#130] Keys: [] Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] @@ -1176,7 +1176,7 @@ Results [3]: [sum#341, isEmpty#342, sum#343] Input [3]: [sum#341, isEmpty#342, sum#343] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#344] -(208) HashAggregate [codegen id : 659] +(208) HashAggregate [codegen id : 704] Input [3]: [sum#341, isEmpty#342, sum#343] Keys: [] Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] @@ -1185,7 +1185,7 @@ Results [6]: [null AS channel#347, null AS i_brand_id#348, null AS i_class_id#34 (209) Union -(210) HashAggregate [codegen id : 660] +(210) HashAggregate [codegen id : 705] Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Functions: [] @@ -1196,7 +1196,7 @@ Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sa Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#353] -(212) HashAggregate [codegen id : 661] +(212) HashAggregate [codegen id : 706] Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Functions: [] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index cb186b0ddcd3b..0a9fa7cbf3498 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (661) + WholeStageCodegen (706) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (660) + WholeStageCodegen (705) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (131) + WholeStageCodegen (140) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (130) + WholeStageCodegen (139) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (43) + WholeStageCodegen (46) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -71,14 +71,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (42) + WholeStageCodegen (45) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk,ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [ss_item_sk,ss_item_sk] + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter WholeStageCodegen (2) Sort [ss_item_sk] InputAdapter @@ -90,11 +90,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - WholeStageCodegen (20) + InputAdapter + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #6 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -103,23 +104,23 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -141,44 +142,48 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #12 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + BroadcastExchange #15 + WholeStageCodegen (7) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -194,7 +199,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter BroadcastExchange #5 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -202,46 +207,50 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #17 - SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #18 - WholeStageCodegen (22) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk] #19 - WholeStageCodegen (86) + WholeStageCodegen (44) + SortMergeJoin [i_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (24) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #18 + WholeStageCodegen (23) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (43) + Sort [ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk] #19 + WholeStageCodegen (92) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (85) + WholeStageCodegen (91) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_item_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (45) + SortMergeJoin [cs_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (48) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #24 - WholeStageCodegen (44) + WholeStageCodegen (47) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (63) + InputAdapter + WholeStageCodegen (67) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -249,32 +258,33 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (129) + WholeStageCodegen (138) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #25 - WholeStageCodegen (128) + WholeStageCodegen (137) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk,ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - InputAdapter - SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (88) + SortMergeJoin [ws_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (94) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #26 - WholeStageCodegen (87) + WholeStageCodegen (93) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (106) + InputAdapter + WholeStageCodegen (113) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -282,136 +292,136 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (263) + WholeStageCodegen (281) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id] #27 - WholeStageCodegen (262) + WholeStageCodegen (280) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #28 - WholeStageCodegen (261) + WholeStageCodegen (279) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (174) + WholeStageCodegen (186) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (217) + WholeStageCodegen (232) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (260) + WholeStageCodegen (278) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (395) + WholeStageCodegen (422) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id] #32 - WholeStageCodegen (394) + WholeStageCodegen (421) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #33 - WholeStageCodegen (393) + WholeStageCodegen (420) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (306) + WholeStageCodegen (327) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (349) + WholeStageCodegen (373) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (392) + WholeStageCodegen (419) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (527) + WholeStageCodegen (563) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel] #34 - WholeStageCodegen (526) + WholeStageCodegen (562) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #35 - WholeStageCodegen (525) + WholeStageCodegen (561) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (438) + WholeStageCodegen (468) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (481) + WholeStageCodegen (514) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (524) + WholeStageCodegen (560) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (659) + WholeStageCodegen (704) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange #36 - WholeStageCodegen (658) + WholeStageCodegen (703) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #37 - WholeStageCodegen (657) + WholeStageCodegen (702) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (570) + WholeStageCodegen (609) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (613) + WholeStageCodegen (655) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (656) + WholeStageCodegen (701) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index f7fff421817c0..7c032590fe404 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject (57) : : +- * Filter (34) : : +- SortMergeJoin ExistenceJoin(exists#1) (33) : : :- SortMergeJoin ExistenceJoin(exists#2) (25) - : : : :- SortMergeJoin LeftSemi (17) + : : : :- * SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) @@ -129,7 +129,7 @@ Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(17) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None @@ -141,18 +141,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 8] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#17] -(21) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#17] Join condition: None -(22) Project [codegen id : 7] +(22) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] @@ -160,7 +160,7 @@ Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] -(24) Sort [codegen id : 8] +(24) Sort [codegen id : 9] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 @@ -176,18 +176,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 11] Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#21] -(29) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#20] Right keys [1]: [d_date_sk#21] Join condition: None -(30) Project [codegen id : 10] +(30) Project [codegen id : 11] Output [1]: [cs_ship_customer_sk#19] Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] @@ -195,7 +195,7 @@ Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] Input [1]: [cs_ship_customer_sk#19] Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] -(32) Sort [codegen id : 11] +(32) Sort [codegen id : 12] Input [1]: [cs_ship_customer_sk#19] Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 @@ -204,11 +204,11 @@ Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#19] Join condition: None -(34) Filter [codegen id : 12] +(34) Filter [codegen id : 13] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(35) Project [codegen id : 12] +(35) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -216,7 +216,7 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] -(37) Sort [codegen id : 13] +(37) Sort [codegen id : 14] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 @@ -227,10 +227,10 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 14] +(39) ColumnarToRow [codegen id : 15] Input [2]: [ca_address_sk#24, ca_state#25] -(40) Filter [codegen id : 14] +(40) Filter [codegen id : 15] Input [2]: [ca_address_sk#24, ca_state#25] Condition : isnotnull(ca_address_sk#24) @@ -238,16 +238,16 @@ Condition : isnotnull(ca_address_sk#24) Input [2]: [ca_address_sk#24, ca_state#25] Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] -(42) Sort [codegen id : 15] +(42) Sort [codegen id : 16] Input [2]: [ca_address_sk#24, ca_state#25] Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 16] +(43) SortMergeJoin [codegen id : 17] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#24] Join condition: None -(44) Project [codegen id : 16] +(44) Project [codegen id : 17] Output [2]: [c_current_cdemo_sk#4, ca_state#25] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] @@ -255,7 +255,7 @@ Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_stat Input [2]: [c_current_cdemo_sk#4, ca_state#25] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] -(46) Sort [codegen id : 17] +(46) Sort [codegen id : 18] Input [2]: [c_current_cdemo_sk#4, ca_state#25] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 @@ -266,10 +266,10 @@ Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 18] +(48) ColumnarToRow [codegen id : 19] Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -(49) Filter [codegen id : 18] +(49) Filter [codegen id : 19] Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Condition : isnotnull(cd_demo_sk#28) @@ -277,20 +277,20 @@ Condition : isnotnull(cd_demo_sk#28) Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] -(51) Sort [codegen id : 19] +(51) Sort [codegen id : 20] Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 -(52) SortMergeJoin [codegen id : 20] +(52) SortMergeJoin [codegen id : 21] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#28] Join condition: None -(53) Project [codegen id : 20] +(53) Project [codegen id : 21] Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -(54) HashAggregate [codegen id : 20] +(54) HashAggregate [codegen id : 21] Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#31), partial_max(cd_dep_count#31), partial_sum(cd_dep_count#31), partial_avg(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_sum(cd_dep_employed_count#32), partial_avg(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_sum(cd_dep_college_count#33)] @@ -301,7 +301,7 @@ Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] -(56) HashAggregate [codegen id : 21] +(56) HashAggregate [codegen id : 22] Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [10]: [count(1), avg(cd_dep_count#31), max(cd_dep_count#31), sum(cd_dep_count#31), avg(cd_dep_employed_count#32), max(cd_dep_employed_count#32), sum(cd_dep_employed_count#32), avg(cd_dep_college_count#33), max(cd_dep_college_count#33), sum(cd_dep_college_count#33)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt index ee441a8332a2a..6dc49f8dce3e5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt @@ -1,66 +1,69 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (21) + WholeStageCodegen (22) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (20) + WholeStageCodegen (21) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (18) Sort [c_current_cdemo_sk] InputAdapter Exchange [c_current_cdemo_sk] #2 - WholeStageCodegen (16) + WholeStageCodegen (17) Project [c_current_cdemo_sk,ca_state] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (14) Sort [c_current_addr_sk] InputAdapter Exchange [c_current_addr_sk] #3 - WholeStageCodegen (12) + WholeStageCodegen (13) Project [c_current_cdemo_sk,c_current_addr_sk] Filter [exists,exists] InputAdapter SortMergeJoin [c_customer_sk,cs_ship_customer_sk] SortMergeJoin [c_customer_sk,ws_bill_customer_sk] - SortMergeJoin [c_customer_sk,ss_customer_sk] - WholeStageCodegen (2) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #4 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #5 - WholeStageCodegen (4) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - WholeStageCodegen (8) + WholeStageCodegen (6) + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (2) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #4 + WholeStageCodegen (1) + Filter [c_current_addr_sk,c_current_cdemo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #5 + WholeStageCodegen (4) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] + WholeStageCodegen (9) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #7 - WholeStageCodegen (7) + WholeStageCodegen (8) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -69,11 +72,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (11) + WholeStageCodegen (12) Sort [cs_ship_customer_sk] InputAdapter Exchange [cs_ship_customer_sk] #8 - WholeStageCodegen (10) + WholeStageCodegen (11) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -83,21 +86,21 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter - WholeStageCodegen (15) + WholeStageCodegen (16) Sort [ca_address_sk] InputAdapter Exchange [ca_address_sk] #9 - WholeStageCodegen (14) + WholeStageCodegen (15) Filter [ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - WholeStageCodegen (19) + WholeStageCodegen (20) Sort [cd_demo_sk] InputAdapter Exchange [cd_demo_sk] #10 - WholeStageCodegen (18) + WholeStageCodegen (19) Filter [cd_demo_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index df75bec72326d..8e71d6c5c0965 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -12,8 +12,8 @@ TakeOrderedAndProject (54) : :- * Sort (34) : : +- Exchange (33) : : +- * Project (32) - : : +- SortMergeJoin LeftSemi (31) - : : :- SortMergeJoin LeftSemi (17) + : : +- * SortMergeJoin LeftSemi (31) + : : :- * SortMergeJoin LeftSemi (17) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) @@ -126,7 +126,7 @@ Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(17) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None @@ -138,18 +138,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 8] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join condition: None -(22) Project [codegen id : 7] +(22) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#13 AS customsk#16] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] @@ -160,18 +160,18 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 9] +(24) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (25) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#19] -(26) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#19] Join condition: None -(27) Project [codegen id : 9] +(27) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#17 AS customsk#20] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] @@ -181,16 +181,16 @@ Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] Input [1]: [customsk#16] Arguments: hashpartitioning(customsk#16, 5), ENSURE_REQUIREMENTS, [id=#21] -(30) Sort [codegen id : 10] +(30) Sort [codegen id : 11] Input [1]: [customsk#16] Arguments: [customsk#16 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin +(31) SortMergeJoin [codegen id : 12] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#16] Join condition: None -(32) Project [codegen id : 11] +(32) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -198,7 +198,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#22] -(34) Sort [codegen id : 12] +(34) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_addr_sk#3 ASC NULLS FIRST], false, 0 @@ -209,10 +209,10 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 13] +(36) ColumnarToRow [codegen id : 14] Input [2]: [ca_address_sk#23, ca_state#24] -(37) Filter [codegen id : 13] +(37) Filter [codegen id : 14] Input [2]: [ca_address_sk#23, ca_state#24] Condition : isnotnull(ca_address_sk#23) @@ -220,16 +220,16 @@ Condition : isnotnull(ca_address_sk#23) Input [2]: [ca_address_sk#23, ca_state#24] Arguments: hashpartitioning(ca_address_sk#23, 5), ENSURE_REQUIREMENTS, [id=#25] -(39) Sort [codegen id : 14] +(39) Sort [codegen id : 15] Input [2]: [ca_address_sk#23, ca_state#24] Arguments: [ca_address_sk#23 ASC NULLS FIRST], false, 0 -(40) SortMergeJoin [codegen id : 15] +(40) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#23] Join condition: None -(41) Project [codegen id : 15] +(41) Project [codegen id : 16] Output [2]: [c_current_cdemo_sk#2, ca_state#24] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23, ca_state#24] @@ -237,7 +237,7 @@ Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23, ca_stat Input [2]: [c_current_cdemo_sk#2, ca_state#24] Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#26] -(43) Sort [codegen id : 16] +(43) Sort [codegen id : 17] Input [2]: [c_current_cdemo_sk#2, ca_state#24] Arguments: [c_current_cdemo_sk#2 ASC NULLS FIRST], false, 0 @@ -248,10 +248,10 @@ Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 17] +(45) ColumnarToRow [codegen id : 18] Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -(46) Filter [codegen id : 17] +(46) Filter [codegen id : 18] Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Condition : isnotnull(cd_demo_sk#27) @@ -259,20 +259,20 @@ Condition : isnotnull(cd_demo_sk#27) Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#33] -(48) Sort [codegen id : 18] +(48) Sort [codegen id : 19] Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 -(49) SortMergeJoin [codegen id : 19] +(49) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#27] Join condition: None -(50) Project [codegen id : 19] +(50) Project [codegen id : 20] Output [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Input [8]: [c_current_cdemo_sk#2, ca_state#24, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -(51) HashAggregate [codegen id : 19] +(51) HashAggregate [codegen id : 20] Input [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#30), partial_max(cd_dep_count#30), partial_sum(cd_dep_count#30), partial_avg(cd_dep_employed_count#31), partial_max(cd_dep_employed_count#31), partial_sum(cd_dep_employed_count#31), partial_avg(cd_dep_college_count#32), partial_max(cd_dep_college_count#32), partial_sum(cd_dep_college_count#32)] @@ -283,7 +283,7 @@ Results [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] Arguments: hashpartitioning(ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#60] -(53) HashAggregate [codegen id : 20] +(53) HashAggregate [codegen id : 21] Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [10]: [count(1), avg(cd_dep_count#30), max(cd_dep_count#30), sum(cd_dep_count#30), avg(cd_dep_employed_count#31), max(cd_dep_employed_count#31), sum(cd_dep_employed_count#31), avg(cd_dep_college_count#32), max(cd_dep_college_count#32), sum(cd_dep_college_count#32)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt index 9a090e0da62c8..840b465af0747 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt @@ -1,65 +1,69 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (20) + WholeStageCodegen (21) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (19) + WholeStageCodegen (20) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter - WholeStageCodegen (16) + WholeStageCodegen (17) Sort [c_current_cdemo_sk] InputAdapter Exchange [c_current_cdemo_sk] #2 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [c_current_cdemo_sk,ca_state] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (13) Sort [c_current_addr_sk] InputAdapter Exchange [c_current_addr_sk] #3 - WholeStageCodegen (11) + WholeStageCodegen (12) Project [c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - SortMergeJoin [c_customer_sk,customsk] - SortMergeJoin [c_customer_sk,ss_customer_sk] - WholeStageCodegen (2) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #4 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #5 - WholeStageCodegen (4) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - WholeStageCodegen (10) + SortMergeJoin [c_customer_sk,customsk] + InputAdapter + WholeStageCodegen (6) + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (2) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #4 + WholeStageCodegen (1) + Filter [c_current_addr_sk,c_current_cdemo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #5 + WholeStageCodegen (4) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + WholeStageCodegen (11) Sort [customsk] InputAdapter Exchange [customsk] #7 Union - WholeStageCodegen (7) + WholeStageCodegen (8) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -68,7 +72,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) + WholeStageCodegen (10) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -78,21 +82,21 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter - WholeStageCodegen (14) + WholeStageCodegen (15) Sort [ca_address_sk] InputAdapter Exchange [ca_address_sk] #8 - WholeStageCodegen (13) + WholeStageCodegen (14) Filter [ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - WholeStageCodegen (18) + WholeStageCodegen (19) Sort [cd_demo_sk] InputAdapter Exchange [cd_demo_sk] #9 - WholeStageCodegen (17) + WholeStageCodegen (18) Filter [cd_demo_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 9199a5e51e669..f019e34b60118 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -203,6 +203,28 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession Row(null, null, 6), Row(null, null, 7), Row(null, null, 8), Row(null, null, 9))) } + test("Left Semi SortMergeJoin should be included in WholeStageCodegen") { + val df1 = spark.range(10).select($"id".as("k1")) + val df2 = spark.range(4).select($"id".as("k2")) + val df3 = spark.range(6).select($"id".as("k3")) + + // test one left semi sort merge join + val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_semi") + assert(oneJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(0), Row(1), Row(2), Row(3))) + + // test two left semi sort merge joins + val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") + .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") + assert(twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | + WholeStageCodegenExec(_ : SortMergeJoinExec) => true + }.size === 2) + checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) + } + test("Inner/Cross BroadcastNestedLoopJoinExec should be included in WholeStageCodegen") { val df1 = spark.range(4).select($"id".as("k1")) val df2 = spark.range(3).select($"id".as("k2")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala index 93c5d05da2e8b..3588b9dda90d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -141,7 +141,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSparkSession { } } - test(s"$testName using SortMergeJoin") { + testWithWholeStageCodegenOnAndOff(s"$testName using SortMergeJoin") { _ => extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) =>