[SPARK-34620][SQL] Code-gen broadcast nested loop join (inner/cross)

### What changes were proposed in this pull request?

`BroadcastNestedLoopJoinExec` does not have code-gen, and we can potentially boost the CPU performance for this operator if we add code-gen for it. https://databricks.com/blog/2017/02/16/processing-trillion-rows-per-second-single-machine-can-nested-loop-joins-fast.html also showed the evidence in one fork.

The codegen for `BroadcastNestedLoopJoinExec` shared some code with `HashJoin`, and the interface `JoinCodegenSupport` is created to hold those common logic. This PR is only supporting inner and cross join. Other join types will be added later in followup PRs.

Example query and generated code:

```
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 =!= $"k2").explain("codegen")
```

```
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:203(0.31% used); numInnerClasses:0) ==
*(2) BroadcastNestedLoopJoin BuildRight, Inner, NOT ((k1#2L + 1) = k2#6L)
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4, step=1, splits=2)
+- BroadcastExchange IdentityBroadcastMode, [id=#22]
   +- *(1) Project [id#4L AS k2#6L]
      +- *(1) Range (0, 3, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(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 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 038 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 039 */
/* 040 */       long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 041 */
/* 042 */       long bnlj_value_4 = -1L;
/* 043 */
/* 044 */       bnlj_value_4 = bnlj_expr_0_0 + 1L;
/* 045 */
/* 046 */       boolean bnlj_value_3 = false;
/* 047 */       bnlj_value_3 = bnlj_value_4 == bnlj_value_1;
/* 048 */       boolean bnlj_value_2 = false;
/* 049 */       bnlj_value_2 = !(bnlj_value_3);
/* 050 */       if (!(false || !bnlj_value_2))
/* 051 */       {
/* 052 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 053 */
/* 054 */         range_mutableStateArray_0[3].reset();
/* 055 */
/* 056 */         range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 057 */
/* 058 */         range_mutableStateArray_0[3].write(1, bnlj_value_1);
/* 059 */         append((range_mutableStateArray_0[3].getRow()).copy());
/* 060 */
/* 061 */       }
/* 062 */     }
/* 063 */
/* 064 */   }
/* 065 */
/* 066 */   private void initRange(int idx) {
/* 067 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 068 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(2L);
/* 069 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 070 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 071 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 072 */     long partitionEnd;
/* 073 */
/* 074 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 075 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 076 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 077 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 078 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 079 */     } else {
/* 080 */       range_nextIndex_0 = st.longValue();
/* 081 */     }
/* 082 */     range_batchEnd_0 = range_nextIndex_0;
/* 083 */
/* 084 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 085 */     .multiply(step).add(start);
/* 086 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 087 */       partitionEnd = Long.MAX_VALUE;
/* 088 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 089 */       partitionEnd = Long.MIN_VALUE;
/* 090 */     } else {
/* 091 */       partitionEnd = end.longValue();
/* 092 */     }
/* 093 */
/* 094 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 095 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 096 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 097 */     if (range_numElementsTodo_0 < 0) {
/* 098 */       range_numElementsTodo_0 = 0;
/* 099 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 100 */       range_numElementsTodo_0++;
/* 101 */     }
/* 102 */   }
/* 103 */
/* 104 */   protected void processNext() throws java.io.IOException {
/* 105 */     // initialize Range
/* 106 */     if (!range_initRange_0) {
/* 107 */       range_initRange_0 = true;
/* 108 */       initRange(partitionIndex);
/* 109 */     }
/* 110 */
/* 111 */     while (true) {
/* 112 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 113 */         long range_nextBatchTodo_0;
/* 114 */         if (range_numElementsTodo_0 > 1000L) {
/* 115 */           range_nextBatchTodo_0 = 1000L;
/* 116 */           range_numElementsTodo_0 -= 1000L;
/* 117 */         } else {
/* 118 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 119 */           range_numElementsTodo_0 = 0;
/* 120 */           if (range_nextBatchTodo_0 == 0) break;
/* 121 */         }
/* 122 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 123 */       }
/* 124 */
/* 125 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 126 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 127 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 128 */
/* 129 */         // common sub-expressions
/* 130 */
/* 131 */         bnlj_doConsume_0(range_value_0);
/* 132 */
/* 133 */         if (shouldStop()) {
/* 134 */           range_nextIndex_0 = range_value_0 + 1L;
/* 135 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 136 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 137 */           return;
/* 138 */         }
/* 139 */
/* 140 */       }
/* 141 */       range_nextIndex_0 = range_batchEnd_0;
/* 142 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 143 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 144 */       range_taskContext_0.killTaskIfInterrupted();
/* 145 */     }
/* 146 */   }
/* 147 */
/* 148 */ }
```

### Why are the changes needed?

Improve query CPU performance. Added a micro benchmark query in `JoinBenchmark.scala`.
Saw 1x of run time improvement:

```
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
broadcast nested loop join:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------------------------------------
broadcast nested loop join wholestage off          62922          63052         184          0.3        3000.3       1.0X
broadcast nested loop join wholestage on           30946          30972          26          0.7        1475.6       2.0X
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

* Added unit test in `WholeStageCodegenSuite.scala`, and existing unit tests for `BroadcastNestedLoopJoinExec`.
* Updated golden files for several TCPDS query plans, as whole stage code-gen for `BroadcastNestedLoopJoinExec` is triggered.
* Updated `JoinBenchmark-jdk11-results.txt ` and `JoinBenchmark-results.txt` with new benchmark result. Followed previous benchmark PRs - https://github.com/apache/spark/pull/27078 and https://github.com/apache/spark/pull/26003 to use same type of machine:

```
Amazon AWS EC2
type: r3.xlarge
region: us-west-2 (Oregon)
OS: Linux
```

Closes #31736 from c21/nested-join-exec.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This commit is contained in:
Cheng Su 2021-03-09 11:45:43 +00:00 committed by Wenchen Fan
parent 43b23fd132
commit b5b198516c
36 changed files with 1565 additions and 1386 deletions

View file

@ -2,74 +2,81 @@
Join Benchmark
================================================================================================
OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
Join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Join w long wholestage off 4441 4572 185 4.7 211.8 1.0X
Join w long wholestage on 1409 1500 96 14.9 67.2 3.2X
Join w long wholestage off 3931 3998 95 5.3 187.4 1.0X
Join w long wholestage on 1507 1769 178 13.9 71.9 2.6X
OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
Join w long duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Join w long duplicated wholestage off 5111 5116 7 4.1 243.7 1.0X
Join w long duplicated wholestage on 1493 1518 22 14.0 71.2 3.4X
Join w long duplicated wholestage off 5582 5617 50 3.8 266.2 1.0X
Join w long duplicated wholestage on 1435 1451 19 14.6 68.4 3.9X
OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
Join w 2 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Join w 2 ints wholestage off 171821 171906 121 0.1 8193.0 1.0X
Join w 2 ints wholestage on 166559 166975 263 0.1 7942.1 1.0X
Join w 2 ints wholestage off 171470 171478 11 0.1 8176.3 1.0X
Join w 2 ints wholestage on 166612 166762 123 0.1 7944.7 1.0X
OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
Join w 2 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Join w 2 longs wholestage off 7511 7555 62 2.8 358.2 1.0X
Join w 2 longs wholestage on 3776 4119 232 5.6 180.1 2.0X
Join w 2 longs wholestage off 6065 6093 40 3.5 289.2 1.0X
Join w 2 longs wholestage on 3285 3375 97 6.4 156.7 1.8X
OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
Join w 2 longs duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Join w 2 longs duplicated wholestage off 13563 13617 77 1.5 646.7 1.0X
Join w 2 longs duplicated wholestage on 7947 8053 71 2.6 378.9 1.7X
Join w 2 longs duplicated wholestage off 14969 15027 82 1.4 713.8 1.0X
Join w 2 longs duplicated wholestage on 7902 8151 406 2.7 376.8 1.9X
OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
outer join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
outer join w long wholestage off 3915 3923 12 5.4 186.7 1.0X
outer join w long wholestage on 1421 1461 30 14.8 67.8 2.8X
outer join w long wholestage off 2822 2823 1 7.4 134.6 1.0X
outer join w long wholestage on 1419 1436 19 14.8 67.7 2.0X
OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
semi join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
semi join w long wholestage off 2310 2332 30 9.1 110.2 1.0X
semi join w long wholestage on 835 860 34 25.1 39.8 2.8X
semi join w long wholestage off 1821 1832 15 11.5 86.8 1.0X
semi join w long wholestage on 828 853 36 25.3 39.5 2.2X
OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
sort merge join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
sort merge join wholestage off 1846 1886 56 1.1 880.5 1.0X
sort merge join wholestage on 1402 1654 234 1.5 668.3 1.3X
sort merge join wholestage off 1371 1380 13 1.5 653.7 1.0X
sort merge join wholestage on 1197 1244 37 1.8 570.9 1.1X
OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
sort merge join with duplicates wholestage off 2852 2879 38 0.7 1360.0 1.0X
sort merge join with duplicates wholestage on 2645 2742 156 0.8 1261.0 1.1X
sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------
sort merge join with duplicates wholestage off 1920 1933 20 1.1 915.3 1.0X
sort merge join with duplicates wholestage on 1871 1912 27 1.1 892.0 1.0X
OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join wholestage off 1506 1564 82 2.8 359.1 1.0X
shuffle hash join wholestage on 1303 1330 23 3.2 310.6 1.2X
shuffle hash join wholestage off 1102 1122 28 3.8 262.8 1.0X
shuffle hash join wholestage on 657 674 13 6.4 156.6 1.7X
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
broadcast nested loop join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
broadcast nested loop join wholestage off 62922 63052 184 0.3 3000.3 1.0X
broadcast nested loop join wholestage on 30946 30972 26 0.7 1475.6 2.0X

View file

@ -2,74 +2,81 @@
Join Benchmark
================================================================================================
OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
Join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Join w long wholestage off 4531 4557 37 4.6 216.1 1.0X
Join w long wholestage on 1214 1310 95 17.3 57.9 3.7X
Join w long wholestage off 3717 3779 88 5.6 177.2 1.0X
Join w long wholestage on 1283 1458 109 16.3 61.2 2.9X
OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
Join w long duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Join w long duplicated wholestage off 5200 5239 55 4.0 248.0 1.0X
Join w long duplicated wholestage on 1535 1547 11 13.7 73.2 3.4X
Join w long duplicated wholestage off 5342 5378 50 3.9 254.7 1.0X
Join w long duplicated wholestage on 1559 1594 24 13.5 74.3 3.4X
OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
Join w 2 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Join w 2 ints wholestage off 170776 170795 27 0.1 8143.2 1.0X
Join w 2 ints wholestage on 165134 165183 36 0.1 7874.2 1.0X
Join w 2 ints wholestage off 169393 169454 86 0.1 8077.3 1.0X
Join w 2 ints wholestage on 165629 165648 19 0.1 7897.8 1.0X
OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
Join w 2 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Join w 2 longs wholestage off 6561 6595 48 3.2 312.8 1.0X
Join w 2 longs wholestage on 2999 3070 85 7.0 143.0 2.2X
Join w 2 longs wholestage off 6194 6240 66 3.4 295.3 1.0X
Join w 2 longs wholestage on 3389 3447 55 6.2 161.6 1.8X
OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
Join w 2 longs duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Join w 2 longs duplicated wholestage off 15731 15757 38 1.3 750.1 1.0X
Join w 2 longs duplicated wholestage on 8017 8112 80 2.6 382.3 2.0X
Join w 2 longs duplicated wholestage off 14445 14511 93 1.5 688.8 1.0X
Join w 2 longs duplicated wholestage on 8465 8554 120 2.5 403.6 1.7X
OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
outer join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
outer join w long wholestage off 3573 3577 6 5.9 170.4 1.0X
outer join w long wholestage on 1310 1325 15 16.0 62.5 2.7X
outer join w long wholestage off 2565 2582 23 8.2 122.3 1.0X
outer join w long wholestage on 1323 1339 19 15.8 63.1 1.9X
OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
semi join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
semi join w long wholestage off 1893 1916 33 11.1 90.3 1.0X
semi join w long wholestage on 819 842 30 25.6 39.0 2.3X
semi join w long wholestage off 1754 1761 11 12.0 83.6 1.0X
semi join w long wholestage on 846 859 21 24.8 40.3 2.1X
OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
sort merge join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
sort merge join wholestage off 1302 1312 13 1.6 620.9 1.0X
sort merge join wholestage on 1168 1233 62 1.8 557.0 1.1X
sort merge join wholestage off 1283 1294 16 1.6 611.8 1.0X
sort merge join wholestage on 1185 1210 26 1.8 565.0 1.1X
OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
sort merge join with duplicates wholestage off 1996 2005 12 1.1 951.7 1.0X
sort merge join with duplicates wholestage on 1766 1803 42 1.2 842.0 1.1X
sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------
sort merge join with duplicates wholestage off 1915 1934 27 1.1 913.1 1.0X
sort merge join with duplicates wholestage on 1742 1778 36 1.2 830.7 1.1X
OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join wholestage off 1298 1300 3 3.2 309.6 1.0X
shuffle hash join wholestage on 1201 1210 10 3.5 286.4 1.1X
shuffle hash join wholestage off 1037 1047 14 4.0 247.3 1.0X
shuffle hash join wholestage on 652 668 15 6.4 155.4 1.6X
OpenJDK 64-Bit Server VM 1.8.0_272-b10 on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
broadcast nested loop join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
broadcast nested loop join wholestage off 65666 66140 671 0.3 3131.2 1.0X
broadcast nested loop join wholestage on 29429 29523 64 0.7 1403.3 2.2X

View file

@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.aggregate.HashAggregateExec
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, ShuffledHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
@ -51,6 +51,7 @@ trait CodegenSupport extends SparkPlan {
case _: BroadcastHashJoinExec => "bhj"
case _: ShuffledHashJoinExec => "shj"
case _: SortMergeJoinExec => "smj"
case _: BroadcastNestedLoopJoinExec => "bnlj"
case _: RDDScanExec => "rdd"
case _: DataSourceScanExec => "scan"
case _: InMemoryTableScanExec => "memoryScan"

View file

@ -21,10 +21,11 @@ import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide}
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution.{ExplainUtils, SparkPlan}
import org.apache.spark.sql.execution.{CodegenSupport, ExplainUtils, SparkPlan}
import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.util.collection.{BitSet, CompactBuffer}
@ -33,7 +34,7 @@ case class BroadcastNestedLoopJoinExec(
right: SparkPlan,
buildSide: BuildSide,
joinType: JoinType,
condition: Option[Expression]) extends BaseJoinExec {
condition: Option[Expression]) extends JoinCodegenSupport {
override def leftKeys: Seq[Expression] = Nil
override def rightKeys: Seq[Expression] = Nil
@ -393,4 +394,62 @@ case class BroadcastNestedLoopJoinExec(
}
}
}
override def supportCodegen: Boolean = {
joinType.isInstanceOf[InnerLike]
}
override def inputRDDs(): Seq[RDD[InternalRow]] = {
streamed.asInstanceOf[CodegenSupport].inputRDDs()
}
override def needCopyResult: Boolean = true
override def doProduce(ctx: CodegenContext): String = {
streamed.asInstanceOf[CodegenSupport].produce(ctx, this)
}
override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
joinType match {
case _: InnerLike => codegenInner(ctx, input)
case _ =>
throw new IllegalArgumentException(
s"BroadcastNestedLoopJoin code-gen should not take $joinType as the JoinType")
}
}
/**
* Returns the variable name for [[Broadcast]] side.
*/
private def prepareBroadcast(ctx: CodegenContext): String = {
// Create a name for broadcast side
val broadcastArray = broadcast.executeBroadcast[Array[InternalRow]]()
val broadcastTerm = ctx.addReferenceObj("broadcastTerm", broadcastArray)
// Inline mutable state since not many join operations in a task
ctx.addMutableState("InternalRow[]", "buildRowArray",
v => s"$v = (InternalRow[]) $broadcastTerm.value();", forceInline = true)
}
private def codegenInner(ctx: CodegenContext, input: Seq[ExprCode]): String = {
val buildRowArrayTerm = prepareBroadcast(ctx)
val (buildRow, checkCondition, buildVars) = getJoinCondition(ctx, input, streamed, broadcast)
val resultVars = buildSide match {
case BuildLeft => buildVars ++ input
case BuildRight => input ++ buildVars
}
val arrayIndex = ctx.freshName("arrayIndex")
val numOutput = metricTerm(ctx, "numOutputRows")
s"""
|for (int $arrayIndex = 0; $arrayIndex < $buildRowArrayTerm.length; $arrayIndex++) {
| UnsafeRow $buildRow = (UnsafeRow) $buildRowArrayTerm[$arrayIndex];
| $checkCondition {
| $numOutput.add(1);
| ${consume(ctx, resultVars)}
| }
|}
""".stripMargin
}
}

View file

@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.analysis.CastSupport
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide}
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
@ -40,7 +39,7 @@ private[joins] case class HashedRelationInfo(
keyIsUnique: Boolean,
isEmpty: Boolean)
trait HashJoin extends BaseJoinExec with CodegenSupport {
trait HashJoin extends JoinCodegenSupport {
def buildSide: BuildSide
override def simpleStringWithNodeId(): String = {
@ -378,71 +377,13 @@ trait HashJoin extends BaseJoinExec with CodegenSupport {
}
}
/**
* Generates the code for variable of build side.
*/
private def genBuildSideVars(ctx: CodegenContext, matched: String): Seq[ExprCode] = {
ctx.currentVars = null
ctx.INPUT_ROW = matched
buildPlan.output.zipWithIndex.map { case (a, i) =>
val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx)
if (joinType.isInstanceOf[InnerLike]) {
ev
} else {
// the variables are needed even there is no matched rows
val isNull = ctx.freshName("isNull")
val value = ctx.freshName("value")
val javaType = CodeGenerator.javaType(a.dataType)
val code = code"""
|boolean $isNull = true;
|$javaType $value = ${CodeGenerator.defaultValue(a.dataType)};
|if ($matched != null) {
| ${ev.code}
| $isNull = ${ev.isNull};
| $value = ${ev.value};
|}
""".stripMargin
ExprCode(code, JavaCode.isNullVariable(isNull), JavaCode.variable(value, a.dataType))
}
}
}
/**
* Generate the (non-equi) condition used to filter joined rows. This is used in Inner, Left Semi
* and Left Anti joins.
*/
protected def getJoinCondition(
ctx: CodegenContext,
input: Seq[ExprCode]): (String, String, Seq[ExprCode]) = {
val matched = ctx.freshName("matched")
val buildVars = genBuildSideVars(ctx, matched)
val checkCondition = if (condition.isDefined) {
val expr = condition.get
// evaluate the variables from build side that used by condition
val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references)
// filter the output via condition
ctx.currentVars = input ++ buildVars
val ev =
BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx)
val skipRow = s"${ev.isNull} || !${ev.value}"
s"""
|$eval
|${ev.code}
|if (!($skipRow))
""".stripMargin
} else {
""
}
(matched, checkCondition, buildVars)
}
/**
* Generates the code for Inner join.
*/
protected def codegenInner(ctx: CodegenContext, input: Seq[ExprCode]): String = {
val HashedRelationInfo(relationTerm, keyIsUnique, isEmptyHashedRelation) = prepareRelation(ctx)
val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
val (matched, checkCondition, buildVars) = getJoinCondition(ctx, input)
val (matched, checkCondition, buildVars) = getJoinCondition(ctx, input, streamedPlan, buildPlan)
val numOutput = metricTerm(ctx, "numOutputRows")
val resultVars = buildSide match {
@ -497,7 +438,7 @@ trait HashJoin extends BaseJoinExec with CodegenSupport {
val HashedRelationInfo(relationTerm, keyIsUnique, _) = prepareRelation(ctx)
val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
val matched = ctx.freshName("matched")
val buildVars = genBuildSideVars(ctx, matched)
val buildVars = genBuildSideVars(ctx, matched, buildPlan)
val numOutput = metricTerm(ctx, "numOutputRows")
// filter the output via condition
@ -573,7 +514,7 @@ trait HashJoin extends BaseJoinExec with CodegenSupport {
protected def codegenSemi(ctx: CodegenContext, input: Seq[ExprCode]): String = {
val HashedRelationInfo(relationTerm, keyIsUnique, isEmptyHashedRelation) = prepareRelation(ctx)
val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
val (matched, checkCondition, _) = getJoinCondition(ctx, input)
val (matched, checkCondition, _) = getJoinCondition(ctx, input, streamedPlan, buildPlan)
val numOutput = metricTerm(ctx, "numOutputRows")
if (isEmptyHashedRelation) {
@ -635,7 +576,7 @@ trait HashJoin extends BaseJoinExec with CodegenSupport {
}
val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
val (matched, checkCondition, _) = getJoinCondition(ctx, input)
val (matched, checkCondition, _) = getJoinCondition(ctx, input, streamedPlan, buildPlan)
if (keyIsUnique) {
val found = ctx.freshName("found")
@ -699,7 +640,7 @@ trait HashJoin extends BaseJoinExec with CodegenSupport {
val existsVar = ctx.freshName("exists")
val matched = ctx.freshName("matched")
val buildVars = genBuildSideVars(ctx, matched)
val buildVars = genBuildSideVars(ctx, matched, buildPlan)
val checkCondition = if (condition.isDefined) {
val expr = condition.get
// evaluate the variables from build side that used by condition

View file

@ -0,0 +1,96 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.execution.joins
import org.apache.spark.sql.catalyst.expressions.{BindReferences, BoundReference}
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.plans.InnerLike
import org.apache.spark.sql.execution.{CodegenSupport, SparkPlan}
/**
* An interface for those join physical operators that support codegen.
*/
trait JoinCodegenSupport extends CodegenSupport with BaseJoinExec {
/**
* Generate the (non-equi) condition used to filter joined rows.
* This is used in Inner, Left Semi and Left Anti joins.
*
* @return Tuple of variable name for row of build side, generated code for condition,
* and generated code for variables of build side.
*/
protected def getJoinCondition(
ctx: CodegenContext,
streamVars: Seq[ExprCode],
streamPlan: SparkPlan,
buildPlan: SparkPlan): (String, String, Seq[ExprCode]) = {
val buildRow = ctx.freshName("buildRow")
val buildVars = genBuildSideVars(ctx, buildRow, buildPlan)
val checkCondition = if (condition.isDefined) {
val expr = condition.get
// evaluate the variables from build side that used by condition
val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references)
// filter the output via condition
ctx.currentVars = streamVars ++ buildVars
val ev =
BindReferences.bindReference(expr, streamPlan.output ++ buildPlan.output).genCode(ctx)
val skipRow = s"${ev.isNull} || !${ev.value}"
s"""
|$eval
|${ev.code}
|if (!($skipRow))
""".stripMargin
} else {
""
}
(buildRow, checkCondition, buildVars)
}
/**
* Generates the code for variables of build side.
*/
protected def genBuildSideVars(
ctx: CodegenContext,
buildRow: String,
buildPlan: SparkPlan): Seq[ExprCode] = {
ctx.currentVars = null
ctx.INPUT_ROW = buildRow
buildPlan.output.zipWithIndex.map { case (a, i) =>
val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx)
if (joinType.isInstanceOf[InnerLike]) {
ev
} else {
// the variables are needed even there is no matched rows
val isNull = ctx.freshName("isNull")
val value = ctx.freshName("value")
val javaType = CodeGenerator.javaType(a.dataType)
val code = code"""
|boolean $isNull = true;
|$javaType $value = ${CodeGenerator.defaultValue(a.dataType)};
|if ($buildRow != null) {
| ${ev.code}
| $isNull = ${ev.isNull};
| $value = ${ev.value};
|}
""".stripMargin
ExprCode(code, JavaCode.isNullVariable(isNull), JavaCode.variable(value, a.dataType))
}
}
}
}

View file

@ -1,9 +1,9 @@
== Physical Plan ==
BroadcastNestedLoopJoin Inner BuildRight (70)
:- BroadcastNestedLoopJoin Inner BuildRight (58)
: :- BroadcastNestedLoopJoin Inner BuildRight (46)
: : :- BroadcastNestedLoopJoin Inner BuildRight (34)
: : : :- BroadcastNestedLoopJoin Inner BuildRight (22)
* BroadcastNestedLoopJoin Inner BuildRight (70)
:- * BroadcastNestedLoopJoin Inner BuildRight (58)
: :- * BroadcastNestedLoopJoin Inner BuildRight (46)
: : :- * BroadcastNestedLoopJoin Inner BuildRight (34)
: : : :- * BroadcastNestedLoopJoin Inner BuildRight (22)
: : : : :- * HashAggregate (10)
: : : : : +- Exchange (9)
: : : : : +- * HashAggregate (8)
@ -118,7 +118,7 @@ Results [4]: [sum#8, count#9, count#10, count#13]
Input [4]: [sum#8, count#9, count#10, count#13]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#14]
(10) HashAggregate [codegen id : 3]
(10) HashAggregate [codegen id : 18]
Input [4]: [sum#8, count#9, count#10, count#13]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -132,18 +132,18 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))]
ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
(12) ColumnarToRow [codegen id : 4]
(12) ColumnarToRow [codegen id : 3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(13) Filter [codegen id : 4]
(13) Filter [codegen id : 3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00))))
(14) Project [codegen id : 4]
(14) Project [codegen id : 3]
Output [1]: [ss_list_price#3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(15) HashAggregate [codegen id : 4]
(15) HashAggregate [codegen id : 3]
Input [1]: [ss_list_price#3]
Keys [1]: [ss_list_price#3]
Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
@ -154,14 +154,14 @@ Results [4]: [ss_list_price#3, sum#20, count#21, count#22]
Input [4]: [ss_list_price#3, sum#20, count#21, count#22]
Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23]
(17) HashAggregate [codegen id : 5]
(17) HashAggregate [codegen id : 4]
Input [4]: [ss_list_price#3, sum#20, count#21, count#22]
Keys [1]: [ss_list_price#3]
Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19]
Results [4]: [ss_list_price#3, sum#20, count#21, count#22]
(18) HashAggregate [codegen id : 5]
(18) HashAggregate [codegen id : 4]
Input [4]: [ss_list_price#3, sum#20, count#21, count#22]
Keys: []
Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
@ -172,7 +172,7 @@ Results [4]: [sum#20, count#21, count#22, count#25]
Input [4]: [sum#20, count#21, count#22, count#25]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26]
(20) HashAggregate [codegen id : 6]
(20) HashAggregate [codegen id : 5]
Input [4]: [sum#20, count#21, count#22, count#25]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -183,7 +183,7 @@ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(1
Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29]
Arguments: IdentityBroadcastMode, [id=#30]
(22) BroadcastNestedLoopJoin
(22) BroadcastNestedLoopJoin [codegen id : 18]
Join condition: None
(23) Scan parquet default.store_sales
@ -193,18 +193,18 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))]
ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
(24) ColumnarToRow [codegen id : 7]
(24) ColumnarToRow [codegen id : 6]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(25) Filter [codegen id : 7]
(25) Filter [codegen id : 6]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00))))
(26) Project [codegen id : 7]
(26) Project [codegen id : 6]
Output [1]: [ss_list_price#3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(27) HashAggregate [codegen id : 7]
(27) HashAggregate [codegen id : 6]
Input [1]: [ss_list_price#3]
Keys [1]: [ss_list_price#3]
Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
@ -215,14 +215,14 @@ Results [4]: [ss_list_price#3, sum#33, count#34, count#35]
Input [4]: [ss_list_price#3, sum#33, count#34, count#35]
Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36]
(29) HashAggregate [codegen id : 8]
(29) HashAggregate [codegen id : 7]
Input [4]: [ss_list_price#3, sum#33, count#34, count#35]
Keys [1]: [ss_list_price#3]
Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32]
Results [4]: [ss_list_price#3, sum#33, count#34, count#35]
(30) HashAggregate [codegen id : 8]
(30) HashAggregate [codegen id : 7]
Input [4]: [ss_list_price#3, sum#33, count#34, count#35]
Keys: []
Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
@ -233,7 +233,7 @@ Results [4]: [sum#33, count#34, count#35, count#38]
Input [4]: [sum#33, count#34, count#35, count#38]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39]
(32) HashAggregate [codegen id : 9]
(32) HashAggregate [codegen id : 8]
Input [4]: [sum#33, count#34, count#35, count#38]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -244,7 +244,7 @@ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(1
Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42]
Arguments: IdentityBroadcastMode, [id=#43]
(34) BroadcastNestedLoopJoin
(34) BroadcastNestedLoopJoin [codegen id : 18]
Join condition: None
(35) Scan parquet default.store_sales
@ -254,18 +254,18 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))]
ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
(36) ColumnarToRow [codegen id : 10]
(36) ColumnarToRow [codegen id : 9]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(37) Filter [codegen id : 10]
(37) Filter [codegen id : 9]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00))))
(38) Project [codegen id : 10]
(38) Project [codegen id : 9]
Output [1]: [ss_list_price#3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(39) HashAggregate [codegen id : 10]
(39) HashAggregate [codegen id : 9]
Input [1]: [ss_list_price#3]
Keys [1]: [ss_list_price#3]
Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
@ -276,14 +276,14 @@ Results [4]: [ss_list_price#3, sum#46, count#47, count#48]
Input [4]: [ss_list_price#3, sum#46, count#47, count#48]
Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49]
(41) HashAggregate [codegen id : 11]
(41) HashAggregate [codegen id : 10]
Input [4]: [ss_list_price#3, sum#46, count#47, count#48]
Keys [1]: [ss_list_price#3]
Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45]
Results [4]: [ss_list_price#3, sum#46, count#47, count#48]
(42) HashAggregate [codegen id : 11]
(42) HashAggregate [codegen id : 10]
Input [4]: [ss_list_price#3, sum#46, count#47, count#48]
Keys: []
Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
@ -294,7 +294,7 @@ Results [4]: [sum#46, count#47, count#48, count#51]
Input [4]: [sum#46, count#47, count#48, count#51]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52]
(44) HashAggregate [codegen id : 12]
(44) HashAggregate [codegen id : 11]
Input [4]: [sum#46, count#47, count#48, count#51]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -305,7 +305,7 @@ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(1
Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55]
Arguments: IdentityBroadcastMode, [id=#56]
(46) BroadcastNestedLoopJoin
(46) BroadcastNestedLoopJoin [codegen id : 18]
Join condition: None
(47) Scan parquet default.store_sales
@ -315,18 +315,18 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))]
ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
(48) ColumnarToRow [codegen id : 13]
(48) ColumnarToRow [codegen id : 12]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(49) Filter [codegen id : 13]
(49) Filter [codegen id : 12]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00))))
(50) Project [codegen id : 13]
(50) Project [codegen id : 12]
Output [1]: [ss_list_price#3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(51) HashAggregate [codegen id : 13]
(51) HashAggregate [codegen id : 12]
Input [1]: [ss_list_price#3]
Keys [1]: [ss_list_price#3]
Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
@ -337,14 +337,14 @@ Results [4]: [ss_list_price#3, sum#59, count#60, count#61]
Input [4]: [ss_list_price#3, sum#59, count#60, count#61]
Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62]
(53) HashAggregate [codegen id : 14]
(53) HashAggregate [codegen id : 13]
Input [4]: [ss_list_price#3, sum#59, count#60, count#61]
Keys [1]: [ss_list_price#3]
Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58]
Results [4]: [ss_list_price#3, sum#59, count#60, count#61]
(54) HashAggregate [codegen id : 14]
(54) HashAggregate [codegen id : 13]
Input [4]: [ss_list_price#3, sum#59, count#60, count#61]
Keys: []
Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
@ -355,7 +355,7 @@ Results [4]: [sum#59, count#60, count#61, count#64]
Input [4]: [sum#59, count#60, count#61, count#64]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65]
(56) HashAggregate [codegen id : 15]
(56) HashAggregate [codegen id : 14]
Input [4]: [sum#59, count#60, count#61, count#64]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -366,7 +366,7 @@ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(1
Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68]
Arguments: IdentityBroadcastMode, [id=#69]
(58) BroadcastNestedLoopJoin
(58) BroadcastNestedLoopJoin [codegen id : 18]
Join condition: None
(59) Scan parquet default.store_sales
@ -376,18 +376,18 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))]
ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
(60) ColumnarToRow [codegen id : 16]
(60) ColumnarToRow [codegen id : 15]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(61) Filter [codegen id : 16]
(61) Filter [codegen id : 15]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00))))
(62) Project [codegen id : 16]
(62) Project [codegen id : 15]
Output [1]: [ss_list_price#3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(63) HashAggregate [codegen id : 16]
(63) HashAggregate [codegen id : 15]
Input [1]: [ss_list_price#3]
Keys [1]: [ss_list_price#3]
Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
@ -398,14 +398,14 @@ Results [4]: [ss_list_price#3, sum#72, count#73, count#74]
Input [4]: [ss_list_price#3, sum#72, count#73, count#74]
Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75]
(65) HashAggregate [codegen id : 17]
(65) HashAggregate [codegen id : 16]
Input [4]: [ss_list_price#3, sum#72, count#73, count#74]
Keys [1]: [ss_list_price#3]
Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71]
Results [4]: [ss_list_price#3, sum#72, count#73, count#74]
(66) HashAggregate [codegen id : 17]
(66) HashAggregate [codegen id : 16]
Input [4]: [ss_list_price#3, sum#72, count#73, count#74]
Keys: []
Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
@ -416,7 +416,7 @@ Results [4]: [sum#72, count#73, count#74, count#77]
Input [4]: [sum#72, count#73, count#74, count#77]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78]
(68) HashAggregate [codegen id : 18]
(68) HashAggregate [codegen id : 17]
Input [4]: [sum#72, count#73, count#74, count#77]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -427,6 +427,6 @@ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(1
Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81]
Arguments: IdentityBroadcastMode, [id=#82]
(70) BroadcastNestedLoopJoin
(70) BroadcastNestedLoopJoin [codegen id : 18]
Join condition: None

View file

@ -1,9 +1,9 @@
BroadcastNestedLoopJoin
WholeStageCodegen (18)
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
WholeStageCodegen (3)
BroadcastNestedLoopJoin
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count]
InputAdapter
Exchange #1
@ -19,88 +19,93 @@ BroadcastNestedLoopJoin
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
BroadcastExchange #3
WholeStageCodegen (6)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count]
InputAdapter
BroadcastExchange #3
WholeStageCodegen (5)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count]
InputAdapter
Exchange #4
WholeStageCodegen (4)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #5
WholeStageCodegen (3)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
InputAdapter
BroadcastExchange #6
WholeStageCodegen (8)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count]
InputAdapter
Exchange #7
WholeStageCodegen (7)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #8
WholeStageCodegen (6)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
InputAdapter
BroadcastExchange #9
WholeStageCodegen (11)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count]
InputAdapter
Exchange #4
WholeStageCodegen (5)
Exchange #10
WholeStageCodegen (10)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #5
WholeStageCodegen (4)
Exchange [ss_list_price] #11
WholeStageCodegen (9)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
BroadcastExchange #6
WholeStageCodegen (9)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count]
InputAdapter
BroadcastExchange #12
WholeStageCodegen (14)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count]
InputAdapter
Exchange #7
WholeStageCodegen (8)
Exchange #13
WholeStageCodegen (13)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #8
WholeStageCodegen (7)
Exchange [ss_list_price] #14
WholeStageCodegen (12)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
BroadcastExchange #9
WholeStageCodegen (12)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count]
InputAdapter
BroadcastExchange #15
WholeStageCodegen (17)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count]
InputAdapter
Exchange #10
WholeStageCodegen (11)
Exchange #16
WholeStageCodegen (16)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #11
WholeStageCodegen (10)
Exchange [ss_list_price] #17
WholeStageCodegen (15)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
BroadcastExchange #12
WholeStageCodegen (15)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count]
InputAdapter
Exchange #13
WholeStageCodegen (14)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #14
WholeStageCodegen (13)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
BroadcastExchange #15
WholeStageCodegen (18)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count]
InputAdapter
Exchange #16
WholeStageCodegen (17)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #17
WholeStageCodegen (16)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]

View file

@ -1,9 +1,9 @@
== Physical Plan ==
BroadcastNestedLoopJoin Inner BuildRight (70)
:- BroadcastNestedLoopJoin Inner BuildRight (58)
: :- BroadcastNestedLoopJoin Inner BuildRight (46)
: : :- BroadcastNestedLoopJoin Inner BuildRight (34)
: : : :- BroadcastNestedLoopJoin Inner BuildRight (22)
* BroadcastNestedLoopJoin Inner BuildRight (70)
:- * BroadcastNestedLoopJoin Inner BuildRight (58)
: :- * BroadcastNestedLoopJoin Inner BuildRight (46)
: : :- * BroadcastNestedLoopJoin Inner BuildRight (34)
: : : :- * BroadcastNestedLoopJoin Inner BuildRight (22)
: : : : :- * HashAggregate (10)
: : : : : +- Exchange (9)
: : : : : +- * HashAggregate (8)
@ -118,7 +118,7 @@ Results [4]: [sum#8, count#9, count#10, count#13]
Input [4]: [sum#8, count#9, count#10, count#13]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#14]
(10) HashAggregate [codegen id : 3]
(10) HashAggregate [codegen id : 18]
Input [4]: [sum#8, count#9, count#10, count#13]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -132,18 +132,18 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))]
ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
(12) ColumnarToRow [codegen id : 4]
(12) ColumnarToRow [codegen id : 3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(13) Filter [codegen id : 4]
(13) Filter [codegen id : 3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00))))
(14) Project [codegen id : 4]
(14) Project [codegen id : 3]
Output [1]: [ss_list_price#3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(15) HashAggregate [codegen id : 4]
(15) HashAggregate [codegen id : 3]
Input [1]: [ss_list_price#3]
Keys [1]: [ss_list_price#3]
Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
@ -154,14 +154,14 @@ Results [4]: [ss_list_price#3, sum#20, count#21, count#22]
Input [4]: [ss_list_price#3, sum#20, count#21, count#22]
Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23]
(17) HashAggregate [codegen id : 5]
(17) HashAggregate [codegen id : 4]
Input [4]: [ss_list_price#3, sum#20, count#21, count#22]
Keys [1]: [ss_list_price#3]
Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19]
Results [4]: [ss_list_price#3, sum#20, count#21, count#22]
(18) HashAggregate [codegen id : 5]
(18) HashAggregate [codegen id : 4]
Input [4]: [ss_list_price#3, sum#20, count#21, count#22]
Keys: []
Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
@ -172,7 +172,7 @@ Results [4]: [sum#20, count#21, count#22, count#25]
Input [4]: [sum#20, count#21, count#22, count#25]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26]
(20) HashAggregate [codegen id : 6]
(20) HashAggregate [codegen id : 5]
Input [4]: [sum#20, count#21, count#22, count#25]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -183,7 +183,7 @@ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(1
Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29]
Arguments: IdentityBroadcastMode, [id=#30]
(22) BroadcastNestedLoopJoin
(22) BroadcastNestedLoopJoin [codegen id : 18]
Join condition: None
(23) Scan parquet default.store_sales
@ -193,18 +193,18 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))]
ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
(24) ColumnarToRow [codegen id : 7]
(24) ColumnarToRow [codegen id : 6]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(25) Filter [codegen id : 7]
(25) Filter [codegen id : 6]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00))))
(26) Project [codegen id : 7]
(26) Project [codegen id : 6]
Output [1]: [ss_list_price#3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(27) HashAggregate [codegen id : 7]
(27) HashAggregate [codegen id : 6]
Input [1]: [ss_list_price#3]
Keys [1]: [ss_list_price#3]
Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
@ -215,14 +215,14 @@ Results [4]: [ss_list_price#3, sum#33, count#34, count#35]
Input [4]: [ss_list_price#3, sum#33, count#34, count#35]
Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36]
(29) HashAggregate [codegen id : 8]
(29) HashAggregate [codegen id : 7]
Input [4]: [ss_list_price#3, sum#33, count#34, count#35]
Keys [1]: [ss_list_price#3]
Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32]
Results [4]: [ss_list_price#3, sum#33, count#34, count#35]
(30) HashAggregate [codegen id : 8]
(30) HashAggregate [codegen id : 7]
Input [4]: [ss_list_price#3, sum#33, count#34, count#35]
Keys: []
Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
@ -233,7 +233,7 @@ Results [4]: [sum#33, count#34, count#35, count#38]
Input [4]: [sum#33, count#34, count#35, count#38]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39]
(32) HashAggregate [codegen id : 9]
(32) HashAggregate [codegen id : 8]
Input [4]: [sum#33, count#34, count#35, count#38]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -244,7 +244,7 @@ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(1
Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42]
Arguments: IdentityBroadcastMode, [id=#43]
(34) BroadcastNestedLoopJoin
(34) BroadcastNestedLoopJoin [codegen id : 18]
Join condition: None
(35) Scan parquet default.store_sales
@ -254,18 +254,18 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))]
ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
(36) ColumnarToRow [codegen id : 10]
(36) ColumnarToRow [codegen id : 9]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(37) Filter [codegen id : 10]
(37) Filter [codegen id : 9]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00))))
(38) Project [codegen id : 10]
(38) Project [codegen id : 9]
Output [1]: [ss_list_price#3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(39) HashAggregate [codegen id : 10]
(39) HashAggregate [codegen id : 9]
Input [1]: [ss_list_price#3]
Keys [1]: [ss_list_price#3]
Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
@ -276,14 +276,14 @@ Results [4]: [ss_list_price#3, sum#46, count#47, count#48]
Input [4]: [ss_list_price#3, sum#46, count#47, count#48]
Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49]
(41) HashAggregate [codegen id : 11]
(41) HashAggregate [codegen id : 10]
Input [4]: [ss_list_price#3, sum#46, count#47, count#48]
Keys [1]: [ss_list_price#3]
Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45]
Results [4]: [ss_list_price#3, sum#46, count#47, count#48]
(42) HashAggregate [codegen id : 11]
(42) HashAggregate [codegen id : 10]
Input [4]: [ss_list_price#3, sum#46, count#47, count#48]
Keys: []
Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
@ -294,7 +294,7 @@ Results [4]: [sum#46, count#47, count#48, count#51]
Input [4]: [sum#46, count#47, count#48, count#51]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52]
(44) HashAggregate [codegen id : 12]
(44) HashAggregate [codegen id : 11]
Input [4]: [sum#46, count#47, count#48, count#51]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -305,7 +305,7 @@ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(1
Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55]
Arguments: IdentityBroadcastMode, [id=#56]
(46) BroadcastNestedLoopJoin
(46) BroadcastNestedLoopJoin [codegen id : 18]
Join condition: None
(47) Scan parquet default.store_sales
@ -315,18 +315,18 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))]
ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
(48) ColumnarToRow [codegen id : 13]
(48) ColumnarToRow [codegen id : 12]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(49) Filter [codegen id : 13]
(49) Filter [codegen id : 12]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00))))
(50) Project [codegen id : 13]
(50) Project [codegen id : 12]
Output [1]: [ss_list_price#3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(51) HashAggregate [codegen id : 13]
(51) HashAggregate [codegen id : 12]
Input [1]: [ss_list_price#3]
Keys [1]: [ss_list_price#3]
Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
@ -337,14 +337,14 @@ Results [4]: [ss_list_price#3, sum#59, count#60, count#61]
Input [4]: [ss_list_price#3, sum#59, count#60, count#61]
Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62]
(53) HashAggregate [codegen id : 14]
(53) HashAggregate [codegen id : 13]
Input [4]: [ss_list_price#3, sum#59, count#60, count#61]
Keys [1]: [ss_list_price#3]
Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58]
Results [4]: [ss_list_price#3, sum#59, count#60, count#61]
(54) HashAggregate [codegen id : 14]
(54) HashAggregate [codegen id : 13]
Input [4]: [ss_list_price#3, sum#59, count#60, count#61]
Keys: []
Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
@ -355,7 +355,7 @@ Results [4]: [sum#59, count#60, count#61, count#64]
Input [4]: [sum#59, count#60, count#61, count#64]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65]
(56) HashAggregate [codegen id : 15]
(56) HashAggregate [codegen id : 14]
Input [4]: [sum#59, count#60, count#61, count#64]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -366,7 +366,7 @@ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(1
Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68]
Arguments: IdentityBroadcastMode, [id=#69]
(58) BroadcastNestedLoopJoin
(58) BroadcastNestedLoopJoin [codegen id : 18]
Join condition: None
(59) Scan parquet default.store_sales
@ -376,18 +376,18 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))]
ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
(60) ColumnarToRow [codegen id : 16]
(60) ColumnarToRow [codegen id : 15]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(61) Filter [codegen id : 16]
(61) Filter [codegen id : 15]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00))))
(62) Project [codegen id : 16]
(62) Project [codegen id : 15]
Output [1]: [ss_list_price#3]
Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
(63) HashAggregate [codegen id : 16]
(63) HashAggregate [codegen id : 15]
Input [1]: [ss_list_price#3]
Keys [1]: [ss_list_price#3]
Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
@ -398,14 +398,14 @@ Results [4]: [ss_list_price#3, sum#72, count#73, count#74]
Input [4]: [ss_list_price#3, sum#72, count#73, count#74]
Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75]
(65) HashAggregate [codegen id : 17]
(65) HashAggregate [codegen id : 16]
Input [4]: [ss_list_price#3, sum#72, count#73, count#74]
Keys [1]: [ss_list_price#3]
Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71]
Results [4]: [ss_list_price#3, sum#72, count#73, count#74]
(66) HashAggregate [codegen id : 17]
(66) HashAggregate [codegen id : 16]
Input [4]: [ss_list_price#3, sum#72, count#73, count#74]
Keys: []
Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
@ -416,7 +416,7 @@ Results [4]: [sum#72, count#73, count#74, count#77]
Input [4]: [sum#72, count#73, count#74, count#77]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78]
(68) HashAggregate [codegen id : 18]
(68) HashAggregate [codegen id : 17]
Input [4]: [sum#72, count#73, count#74, count#77]
Keys: []
Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
@ -427,6 +427,6 @@ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(1
Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81]
Arguments: IdentityBroadcastMode, [id=#82]
(70) BroadcastNestedLoopJoin
(70) BroadcastNestedLoopJoin [codegen id : 18]
Join condition: None

View file

@ -1,9 +1,9 @@
BroadcastNestedLoopJoin
WholeStageCodegen (18)
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
WholeStageCodegen (3)
BroadcastNestedLoopJoin
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count]
InputAdapter
Exchange #1
@ -19,88 +19,93 @@ BroadcastNestedLoopJoin
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
BroadcastExchange #3
WholeStageCodegen (6)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count]
InputAdapter
BroadcastExchange #3
WholeStageCodegen (5)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count]
InputAdapter
Exchange #4
WholeStageCodegen (4)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #5
WholeStageCodegen (3)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
InputAdapter
BroadcastExchange #6
WholeStageCodegen (8)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count]
InputAdapter
Exchange #7
WholeStageCodegen (7)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #8
WholeStageCodegen (6)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
InputAdapter
BroadcastExchange #9
WholeStageCodegen (11)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count]
InputAdapter
Exchange #4
WholeStageCodegen (5)
Exchange #10
WholeStageCodegen (10)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #5
WholeStageCodegen (4)
Exchange [ss_list_price] #11
WholeStageCodegen (9)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
BroadcastExchange #6
WholeStageCodegen (9)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count]
InputAdapter
BroadcastExchange #12
WholeStageCodegen (14)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count]
InputAdapter
Exchange #7
WholeStageCodegen (8)
Exchange #13
WholeStageCodegen (13)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #8
WholeStageCodegen (7)
Exchange [ss_list_price] #14
WholeStageCodegen (12)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
BroadcastExchange #9
WholeStageCodegen (12)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count]
InputAdapter
BroadcastExchange #15
WholeStageCodegen (17)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count]
InputAdapter
Exchange #10
WholeStageCodegen (11)
Exchange #16
WholeStageCodegen (16)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #11
WholeStageCodegen (10)
Exchange [ss_list_price] #17
WholeStageCodegen (15)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
BroadcastExchange #12
WholeStageCodegen (15)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count]
InputAdapter
Exchange #13
WholeStageCodegen (14)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #14
WholeStageCodegen (13)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
BroadcastExchange #15
WholeStageCodegen (18)
HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count]
InputAdapter
Exchange #16
WholeStageCodegen (17)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
InputAdapter
Exchange [ss_list_price] #17
WholeStageCodegen (16)
HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
Project [ss_list_price]
Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]

View file

@ -1,7 +1,7 @@
== Physical Plan ==
* Sort (69)
+- * Project (68)
+- BroadcastNestedLoopJoin Inner BuildRight (67)
+- * BroadcastNestedLoopJoin Inner BuildRight (67)
:- * HashAggregate (47)
: +- Exchange (46)
: +- * HashAggregate (45)
@ -278,7 +278,7 @@ Results [1]: [sum#30]
Input [1]: [sum#30]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31]
(47) HashAggregate [codegen id : 8]
(47) HashAggregate [codegen id : 15]
Input [1]: [sum#30]
Keys: []
Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))]
@ -293,62 +293,62 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol
PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ext_sales_price:decimal(7,2)>
(49) ColumnarToRow [codegen id : 14]
(49) ColumnarToRow [codegen id : 13]
Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6]
(50) Filter [codegen id : 14]
(50) Filter [codegen id : 13]
Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6]
Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1))
(51) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#8]
(52) BroadcastHashJoin [codegen id : 14]
(52) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_sold_date_sk#6]
Right keys [1]: [d_date_sk#8]
Join condition: None
(53) Project [codegen id : 14]
(53) Project [codegen id : 13]
Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5]
Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#8]
(54) ReusedExchange [Reuses operator id: 15]
Output [1]: [i_item_sk#12]
(55) BroadcastHashJoin [codegen id : 14]
(55) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [i_item_sk#12]
Join condition: None
(56) Project [codegen id : 14]
(56) Project [codegen id : 13]
Output [3]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5]
Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, i_item_sk#12]
(57) ReusedExchange [Reuses operator id: 29]
Output [1]: [s_store_sk#20]
(58) BroadcastHashJoin [codegen id : 14]
(58) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#20]
Join condition: None
(59) Project [codegen id : 14]
(59) Project [codegen id : 13]
Output [2]: [ss_customer_sk#2, ss_ext_sales_price#5]
Input [4]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, s_store_sk#20]
(60) ReusedExchange [Reuses operator id: 42]
Output [1]: [c_customer_sk#23]
(61) BroadcastHashJoin [codegen id : 14]
(61) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_customer_sk#2]
Right keys [1]: [c_customer_sk#23]
Join condition: None
(62) Project [codegen id : 14]
(62) Project [codegen id : 13]
Output [1]: [ss_ext_sales_price#5]
Input [3]: [ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#23]
(63) HashAggregate [codegen id : 14]
(63) HashAggregate [codegen id : 13]
Input [1]: [ss_ext_sales_price#5]
Keys: []
Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))]
@ -359,7 +359,7 @@ Results [1]: [sum#35]
Input [1]: [sum#35]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36]
(65) HashAggregate [codegen id : 15]
(65) HashAggregate [codegen id : 14]
Input [1]: [sum#35]
Keys: []
Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))]
@ -370,14 +370,14 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS t
Input [1]: [total#38]
Arguments: IdentityBroadcastMode, [id=#39]
(67) BroadcastNestedLoopJoin
(67) BroadcastNestedLoopJoin [codegen id : 15]
Join condition: None
(68) Project [codegen id : 16]
(68) Project [codegen id : 15]
Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40]
Input [2]: [promotions#33, total#38]
(69) Sort [codegen id : 16]
(69) Sort [codegen id : 15]
Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40]
Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0

View file

@ -1,85 +1,84 @@
WholeStageCodegen (16)
WholeStageCodegen (15)
Sort [promotions,total]
Project [promotions,total]
InputAdapter
BroadcastNestedLoopJoin
WholeStageCodegen (8)
HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum]
InputAdapter
Exchange #1
WholeStageCodegen (7)
HashAggregate [ss_ext_sales_price] [sum,sum]
Project [ss_ext_sales_price]
BroadcastHashJoin [ss_customer_sk,c_customer_sk]
Project [ss_customer_sk,ss_ext_sales_price]
BroadcastHashJoin [ss_store_sk,s_store_sk]
Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price]
BroadcastHashJoin [ss_promo_sk,p_promo_sk]
Project [ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price]
BroadcastHashJoin [ss_item_sk,i_item_sk]
Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price]
BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
Filter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
ReusedExchange [d_date_sk] #2
InputAdapter
BroadcastExchange #2
WholeStageCodegen (1)
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
BroadcastExchange #3
WholeStageCodegen (2)
Project [i_item_sk]
Filter [i_category,i_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.item [i_item_sk,i_category]
InputAdapter
BroadcastExchange #4
WholeStageCodegen (3)
Project [p_promo_sk]
Filter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk]
ColumnarToRow
InputAdapter
Scan parquet default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (4)
Project [s_store_sk]
Filter [s_gmt_offset,s_store_sk]
BroadcastNestedLoopJoin
HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum]
InputAdapter
Exchange #1
WholeStageCodegen (7)
HashAggregate [ss_ext_sales_price] [sum,sum]
Project [ss_ext_sales_price]
BroadcastHashJoin [ss_customer_sk,c_customer_sk]
Project [ss_customer_sk,ss_ext_sales_price]
BroadcastHashJoin [ss_store_sk,s_store_sk]
Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price]
BroadcastHashJoin [ss_promo_sk,p_promo_sk]
Project [ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price]
BroadcastHashJoin [ss_item_sk,i_item_sk]
Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price]
BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
Filter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store [s_store_sk,s_gmt_offset]
Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
ReusedExchange [d_date_sk] #2
InputAdapter
BroadcastExchange #2
WholeStageCodegen (1)
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
BroadcastExchange #3
WholeStageCodegen (2)
Project [i_item_sk]
Filter [i_category,i_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.item [i_item_sk,i_category]
InputAdapter
BroadcastExchange #4
WholeStageCodegen (3)
Project [p_promo_sk]
Filter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk]
ColumnarToRow
InputAdapter
Scan parquet default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv]
InputAdapter
BroadcastExchange #6
WholeStageCodegen (6)
Project [c_customer_sk]
BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
Filter [c_customer_sk,c_current_addr_sk]
ColumnarToRow
InputAdapter
Scan parquet default.customer [c_customer_sk,c_current_addr_sk]
BroadcastExchange #5
WholeStageCodegen (4)
Project [s_store_sk]
Filter [s_gmt_offset,s_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store [s_store_sk,s_gmt_offset]
InputAdapter
BroadcastExchange #6
WholeStageCodegen (6)
Project [c_customer_sk]
BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
Filter [c_customer_sk,c_current_addr_sk]
ColumnarToRow
InputAdapter
BroadcastExchange #7
WholeStageCodegen (5)
Project [ca_address_sk]
Filter [ca_gmt_offset,ca_address_sk]
ColumnarToRow
InputAdapter
Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset]
Scan parquet default.customer [c_customer_sk,c_current_addr_sk]
InputAdapter
BroadcastExchange #7
WholeStageCodegen (5)
Project [ca_address_sk]
Filter [ca_gmt_offset,ca_address_sk]
ColumnarToRow
InputAdapter
Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (15)
WholeStageCodegen (14)
HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum]
InputAdapter
Exchange #9
WholeStageCodegen (14)
WholeStageCodegen (13)
HashAggregate [ss_ext_sales_price] [sum,sum]
Project [ss_ext_sales_price]
BroadcastHashJoin [ss_customer_sk,c_customer_sk]

View file

@ -1,7 +1,7 @@
== Physical Plan ==
* Sort (72)
+- * Project (71)
+- BroadcastNestedLoopJoin Inner BuildRight (70)
+- * BroadcastNestedLoopJoin Inner BuildRight (70)
:- * HashAggregate (47)
: +- Exchange (46)
: +- * HashAggregate (45)
@ -281,7 +281,7 @@ Results [1]: [sum#30]
Input [1]: [sum#30]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31]
(47) HashAggregate [codegen id : 8]
(47) HashAggregate [codegen id : 15]
Input [1]: [sum#30]
Keys: []
Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))]
@ -296,74 +296,74 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol
PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ext_sales_price:decimal(7,2)>
(49) ColumnarToRow [codegen id : 14]
(49) ColumnarToRow [codegen id : 13]
Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6]
(50) Filter [codegen id : 14]
(50) Filter [codegen id : 13]
Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6]
Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1))
(51) ReusedExchange [Reuses operator id: 8]
Output [1]: [s_store_sk#8]
(52) BroadcastHashJoin [codegen id : 14]
(52) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#8]
Join condition: None
(53) Project [codegen id : 14]
(53) Project [codegen id : 13]
Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6]
Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8]
(54) ReusedExchange [Reuses operator id: 22]
Output [1]: [d_date_sk#16]
(55) BroadcastHashJoin [codegen id : 14]
(55) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_sold_date_sk#6]
Right keys [1]: [d_date_sk#16]
Join condition: None
(56) Project [codegen id : 14]
(56) Project [codegen id : 13]
Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5]
Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#16]
(57) ReusedExchange [Reuses operator id: 28]
Output [2]: [c_customer_sk#20, c_current_addr_sk#21]
(58) BroadcastHashJoin [codegen id : 14]
(58) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_customer_sk#2]
Right keys [1]: [c_customer_sk#20]
Join condition: None
(59) Project [codegen id : 14]
(59) Project [codegen id : 13]
Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21]
Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#20, c_current_addr_sk#21]
(60) ReusedExchange [Reuses operator id: 35]
Output [1]: [ca_address_sk#23]
(61) BroadcastHashJoin [codegen id : 14]
(61) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [c_current_addr_sk#21]
Right keys [1]: [ca_address_sk#23]
Join condition: None
(62) Project [codegen id : 14]
(62) Project [codegen id : 13]
Output [2]: [ss_item_sk#1, ss_ext_sales_price#5]
Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21, ca_address_sk#23]
(63) ReusedExchange [Reuses operator id: 42]
Output [1]: [i_item_sk#26]
(64) BroadcastHashJoin [codegen id : 14]
(64) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [i_item_sk#26]
Join condition: None
(65) Project [codegen id : 14]
(65) Project [codegen id : 13]
Output [1]: [ss_ext_sales_price#5]
Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#26]
(66) HashAggregate [codegen id : 14]
(66) HashAggregate [codegen id : 13]
Input [1]: [ss_ext_sales_price#5]
Keys: []
Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))]
@ -374,7 +374,7 @@ Results [1]: [sum#35]
Input [1]: [sum#35]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36]
(68) HashAggregate [codegen id : 15]
(68) HashAggregate [codegen id : 14]
Input [1]: [sum#35]
Keys: []
Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))]
@ -385,14 +385,14 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS t
Input [1]: [total#38]
Arguments: IdentityBroadcastMode, [id=#39]
(70) BroadcastNestedLoopJoin
(70) BroadcastNestedLoopJoin [codegen id : 15]
Join condition: None
(71) Project [codegen id : 16]
(71) Project [codegen id : 15]
Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40]
Input [2]: [promotions#33, total#38]
(72) Sort [codegen id : 16]
(72) Sort [codegen id : 15]
Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40]
Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0

View file

@ -1,85 +1,84 @@
WholeStageCodegen (16)
WholeStageCodegen (15)
Sort [promotions,total]
Project [promotions,total]
InputAdapter
BroadcastNestedLoopJoin
WholeStageCodegen (8)
HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum]
InputAdapter
Exchange #1
WholeStageCodegen (7)
HashAggregate [ss_ext_sales_price] [sum,sum]
Project [ss_ext_sales_price]
BroadcastHashJoin [ss_item_sk,i_item_sk]
Project [ss_item_sk,ss_ext_sales_price]
BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk]
BroadcastHashJoin [ss_customer_sk,c_customer_sk]
Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price]
BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
BroadcastHashJoin [ss_promo_sk,p_promo_sk]
Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
BroadcastHashJoin [ss_store_sk,s_store_sk]
Filter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
ReusedExchange [d_date_sk] #2
BroadcastNestedLoopJoin
HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum]
InputAdapter
Exchange #1
WholeStageCodegen (7)
HashAggregate [ss_ext_sales_price] [sum,sum]
Project [ss_ext_sales_price]
BroadcastHashJoin [ss_item_sk,i_item_sk]
Project [ss_item_sk,ss_ext_sales_price]
BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk]
BroadcastHashJoin [ss_customer_sk,c_customer_sk]
Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price]
BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
BroadcastHashJoin [ss_promo_sk,p_promo_sk]
Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
BroadcastHashJoin [ss_store_sk,s_store_sk]
Filter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk]
ColumnarToRow
InputAdapter
BroadcastExchange #3
WholeStageCodegen (1)
Project [s_store_sk]
Filter [s_gmt_offset,s_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store [s_store_sk,s_gmt_offset]
Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
ReusedExchange [d_date_sk] #2
InputAdapter
BroadcastExchange #4
WholeStageCodegen (2)
Project [p_promo_sk]
Filter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk]
BroadcastExchange #3
WholeStageCodegen (1)
Project [s_store_sk]
Filter [s_gmt_offset,s_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv]
Scan parquet default.store [s_store_sk,s_gmt_offset]
InputAdapter
BroadcastExchange #2
WholeStageCodegen (3)
Project [d_date_sk]
Filter [d_year,d_moy,d_date_sk]
BroadcastExchange #4
WholeStageCodegen (2)
Project [p_promo_sk]
Filter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk]
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
Scan parquet default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (4)
Filter [c_customer_sk,c_current_addr_sk]
ColumnarToRow
InputAdapter
Scan parquet default.customer [c_customer_sk,c_current_addr_sk]
BroadcastExchange #2
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
BroadcastExchange #6
WholeStageCodegen (5)
Project [ca_address_sk]
Filter [ca_gmt_offset,ca_address_sk]
ColumnarToRow
InputAdapter
Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset]
BroadcastExchange #5
WholeStageCodegen (4)
Filter [c_customer_sk,c_current_addr_sk]
ColumnarToRow
InputAdapter
Scan parquet default.customer [c_customer_sk,c_current_addr_sk]
InputAdapter
BroadcastExchange #7
WholeStageCodegen (6)
Project [i_item_sk]
Filter [i_category,i_item_sk]
BroadcastExchange #6
WholeStageCodegen (5)
Project [ca_address_sk]
Filter [ca_gmt_offset,ca_address_sk]
ColumnarToRow
InputAdapter
Scan parquet default.item [i_item_sk,i_category]
Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset]
InputAdapter
BroadcastExchange #7
WholeStageCodegen (6)
Project [i_item_sk]
Filter [i_category,i_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.item [i_item_sk,i_category]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (15)
WholeStageCodegen (14)
HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum]
InputAdapter
Exchange #9
WholeStageCodegen (14)
WholeStageCodegen (13)
HashAggregate [ss_ext_sales_price] [sum,sum]
Project [ss_ext_sales_price]
BroadcastHashJoin [ss_item_sk,i_item_sk]

View file

@ -40,7 +40,7 @@ TakeOrderedAndProject (89)
: : +- ReusedExchange (23)
: +- ReusedExchange (26)
:- * Project (53)
: +- BroadcastNestedLoopJoin Inner BuildRight (52)
: +- * BroadcastNestedLoopJoin Inner BuildRight (52)
: :- * HashAggregate (42)
: : +- Exchange (41)
: : +- * HashAggregate (40)
@ -117,7 +117,7 @@ Input [2]: [d_date_sk#6, d_date#7]
(6) Filter [codegen id : 1]
Input [2]: [d_date_sk#6, d_date#7]
Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 11172)) AND (d_date#7 <= 11202)) AND isnotnull(d_date_sk#6))
Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6))
(7) Project [codegen id : 1]
Output [1]: [d_date_sk#6]
@ -284,7 +284,7 @@ Results [3]: [cs_call_center_sk#40, sum#46, sum#47]
Input [3]: [cs_call_center_sk#40, sum#46, sum#47]
Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48]
(42) HashAggregate [codegen id : 11]
(42) HashAggregate [codegen id : 14]
Input [3]: [cs_call_center_sk#40, sum#46, sum#47]
Keys [1]: [cs_call_center_sk#40]
Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))]
@ -298,22 +298,22 @@ Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)]
ReadSchema: struct<cr_return_amount:decimal(7,2),cr_net_loss:decimal(7,2)>
(44) ColumnarToRow [codegen id : 13]
(44) ColumnarToRow [codegen id : 12]
Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55]
(45) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#6]
(46) BroadcastHashJoin [codegen id : 13]
(46) BroadcastHashJoin [codegen id : 12]
Left keys [1]: [cr_returned_date_sk#55]
Right keys [1]: [d_date_sk#6]
Join condition: None
(47) Project [codegen id : 13]
(47) Project [codegen id : 12]
Output [2]: [cr_return_amount#53, cr_net_loss#54]
Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6]
(48) HashAggregate [codegen id : 13]
(48) HashAggregate [codegen id : 12]
Input [2]: [cr_return_amount#53, cr_net_loss#54]
Keys: []
Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))]
@ -324,7 +324,7 @@ Results [2]: [sum#58, sum#59]
Input [2]: [sum#58, sum#59]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60]
(50) HashAggregate [codegen id : 14]
(50) HashAggregate [codegen id : 13]
Input [2]: [sum#58, sum#59]
Keys: []
Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))]
@ -335,10 +335,10 @@ Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS re
Input [2]: [returns#63, profit_loss#64]
Arguments: IdentityBroadcastMode, [id=#65]
(52) BroadcastNestedLoopJoin
(52) BroadcastNestedLoopJoin [codegen id : 14]
Join condition: None
(53) Project [codegen id : 15]
(53) Project [codegen id : 14]
Output [5]: [sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68]
Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64]
@ -350,22 +350,22 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_so
PushedFilters: [IsNotNull(ws_web_page_sk)]
ReadSchema: struct<ws_web_page_sk:int,ws_ext_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)>
(55) ColumnarToRow [codegen id : 18]
(55) ColumnarToRow [codegen id : 17]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72]
(56) Filter [codegen id : 18]
(56) Filter [codegen id : 17]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72]
Condition : isnotnull(ws_web_page_sk#69)
(57) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#6]
(58) BroadcastHashJoin [codegen id : 18]
(58) BroadcastHashJoin [codegen id : 17]
Left keys [1]: [ws_sold_date_sk#72]
Right keys [1]: [d_date_sk#6]
Join condition: None
(59) Project [codegen id : 18]
(59) Project [codegen id : 17]
Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71]
Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6]
@ -376,10 +376,10 @@ Location [not included in comparison]/{warehouse_dir}/web_page]
PushedFilters: [IsNotNull(wp_web_page_sk)]
ReadSchema: struct<wp_web_page_sk:int>
(61) ColumnarToRow [codegen id : 17]
(61) ColumnarToRow [codegen id : 16]
Input [1]: [wp_web_page_sk#73]
(62) Filter [codegen id : 17]
(62) Filter [codegen id : 16]
Input [1]: [wp_web_page_sk#73]
Condition : isnotnull(wp_web_page_sk#73)
@ -387,16 +387,16 @@ Condition : isnotnull(wp_web_page_sk#73)
Input [1]: [wp_web_page_sk#73]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74]
(64) BroadcastHashJoin [codegen id : 18]
(64) BroadcastHashJoin [codegen id : 17]
Left keys [1]: [ws_web_page_sk#69]
Right keys [1]: [wp_web_page_sk#73]
Join condition: None
(65) Project [codegen id : 18]
(65) Project [codegen id : 17]
Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
(66) HashAggregate [codegen id : 18]
(66) HashAggregate [codegen id : 17]
Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
Keys [1]: [wp_web_page_sk#73]
Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))]
@ -407,7 +407,7 @@ Results [3]: [wp_web_page_sk#73, sum#77, sum#78]
Input [3]: [wp_web_page_sk#73, sum#77, sum#78]
Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79]
(68) HashAggregate [codegen id : 23]
(68) HashAggregate [codegen id : 22]
Input [3]: [wp_web_page_sk#73, sum#77, sum#78]
Keys [1]: [wp_web_page_sk#73]
Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))]
@ -422,38 +422,38 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(w
PushedFilters: [IsNotNull(wr_web_page_sk)]
ReadSchema: struct<wr_web_page_sk:bigint,wr_return_amt:decimal(7,2),wr_net_loss:decimal(7,2)>
(70) ColumnarToRow [codegen id : 21]
(70) ColumnarToRow [codegen id : 20]
Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87]
(71) Filter [codegen id : 21]
(71) Filter [codegen id : 20]
Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87]
Condition : isnotnull(wr_web_page_sk#84)
(72) ReusedExchange [Reuses operator id: 63]
Output [1]: [wp_web_page_sk#88]
(73) BroadcastHashJoin [codegen id : 21]
(73) BroadcastHashJoin [codegen id : 20]
Left keys [1]: [wr_web_page_sk#84]
Right keys [1]: [cast(wp_web_page_sk#88 as bigint)]
Join condition: None
(74) Project [codegen id : 21]
(74) Project [codegen id : 20]
Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88]
Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88]
(75) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#6]
(76) BroadcastHashJoin [codegen id : 21]
(76) BroadcastHashJoin [codegen id : 20]
Left keys [1]: [wr_returned_date_sk#87]
Right keys [1]: [cast(d_date_sk#6 as bigint)]
Join condition: None
(77) Project [codegen id : 21]
(77) Project [codegen id : 20]
Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88]
Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6]
(78) HashAggregate [codegen id : 21]
(78) HashAggregate [codegen id : 20]
Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88]
Keys [1]: [wp_web_page_sk#88]
Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))]
@ -464,7 +464,7 @@ Results [3]: [wp_web_page_sk#88, sum#91, sum#92]
Input [3]: [wp_web_page_sk#88, sum#91, sum#92]
Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93]
(80) HashAggregate [codegen id : 22]
(80) HashAggregate [codegen id : 21]
Input [3]: [wp_web_page_sk#88, sum#91, sum#92]
Keys [1]: [wp_web_page_sk#88]
Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))]
@ -475,22 +475,22 @@ Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85)
Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98]
(82) BroadcastHashJoin [codegen id : 23]
(82) BroadcastHashJoin [codegen id : 22]
Left keys [1]: [wp_web_page_sk#73]
Right keys [1]: [wp_web_page_sk#88]
Join condition: None
(83) Project [codegen id : 23]
(83) Project [codegen id : 22]
Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102]
Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97]
(84) Union
(85) Expand [codegen id : 24]
(85) Expand [codegen id : 23]
Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39]
Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105]
(86) HashAggregate [codegen id : 24]
(86) HashAggregate [codegen id : 23]
Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105]
Keys [3]: [channel#103, id#104, spark_grouping_id#105]
Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)]
@ -501,7 +501,7 @@ Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113,
Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117]
Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118]
(88) HashAggregate [codegen id : 25]
(88) HashAggregate [codegen id : 24]
Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117]
Keys [3]: [channel#103, id#104, spark_grouping_id#105]
Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)]

View file

@ -1,9 +1,9 @@
TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (25)
WholeStageCodegen (24)
HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
Exchange [channel,id,spark_grouping_id] #1
WholeStageCodegen (24)
WholeStageCodegen (23)
HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
Expand [sales,returns,profit,channel,id]
InputAdapter
@ -63,30 +63,29 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedExchange [s_store_sk] #4
InputAdapter
ReusedExchange [d_date_sk] #3
WholeStageCodegen (15)
WholeStageCodegen (14)
Project [sales,returns,profit,profit_loss,cs_call_center_sk]
InputAdapter
BroadcastNestedLoopJoin
WholeStageCodegen (11)
HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum]
InputAdapter
Exchange [cs_call_center_sk] #7
WholeStageCodegen (10)
HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum]
Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #3
BroadcastNestedLoopJoin
HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum]
InputAdapter
Exchange [cs_call_center_sk] #7
WholeStageCodegen (10)
HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum]
Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #3
InputAdapter
BroadcastExchange #8
WholeStageCodegen (14)
WholeStageCodegen (13)
HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum]
InputAdapter
Exchange #9
WholeStageCodegen (13)
WholeStageCodegen (12)
HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum]
Project [cr_return_amount,cr_net_loss]
BroadcastHashJoin [cr_returned_date_sk,d_date_sk]
@ -96,13 +95,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #3
WholeStageCodegen (23)
WholeStageCodegen (22)
Project [sales,returns,profit,profit_loss,wp_web_page_sk]
BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk]
HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum]
InputAdapter
Exchange [wp_web_page_sk] #10
WholeStageCodegen (18)
WholeStageCodegen (17)
HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum]
Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
@ -117,18 +116,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedExchange [d_date_sk] #3
InputAdapter
BroadcastExchange #11
WholeStageCodegen (17)
WholeStageCodegen (16)
Filter [wp_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_page [wp_web_page_sk]
InputAdapter
BroadcastExchange #12
WholeStageCodegen (22)
WholeStageCodegen (21)
HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum]
InputAdapter
Exchange [wp_web_page_sk] #13
WholeStageCodegen (21)
WholeStageCodegen (20)
HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum]
Project [wr_return_amt,wr_net_loss,wp_web_page_sk]
BroadcastHashJoin [wr_returned_date_sk,d_date_sk]

View file

@ -40,7 +40,7 @@ TakeOrderedAndProject (89)
: : +- ReusedExchange (23)
: +- ReusedExchange (26)
:- * Project (53)
: +- BroadcastNestedLoopJoin Inner BuildLeft (52)
: +- * BroadcastNestedLoopJoin Inner BuildLeft (52)
: :- BroadcastExchange (43)
: : +- * HashAggregate (42)
: : +- Exchange (41)
@ -117,7 +117,7 @@ Input [2]: [d_date_sk#6, d_date#7]
(6) Filter [codegen id : 1]
Input [2]: [d_date_sk#6, d_date#7]
Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 11172)) AND (d_date#7 <= 11202)) AND isnotnull(d_date_sk#6))
Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6))
(7) Project [codegen id : 1]
Output [1]: [d_date_sk#6]
@ -328,17 +328,17 @@ Results [2]: [sum#59, sum#60]
Input [2]: [sum#59, sum#60]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61]
(51) HashAggregate [codegen id : 14]
(51) HashAggregate
Input [2]: [sum#59, sum#60]
Keys: []
Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))]
Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63]
Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65]
(52) BroadcastNestedLoopJoin
(52) BroadcastNestedLoopJoin [codegen id : 14]
Join condition: None
(53) Project [codegen id : 15]
(53) Project [codegen id : 14]
Output [5]: [sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68]
Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65]
@ -350,22 +350,22 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_so
PushedFilters: [IsNotNull(ws_web_page_sk)]
ReadSchema: struct<ws_web_page_sk:int,ws_ext_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)>
(55) ColumnarToRow [codegen id : 18]
(55) ColumnarToRow [codegen id : 17]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72]
(56) Filter [codegen id : 18]
(56) Filter [codegen id : 17]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72]
Condition : isnotnull(ws_web_page_sk#69)
(57) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#6]
(58) BroadcastHashJoin [codegen id : 18]
(58) BroadcastHashJoin [codegen id : 17]
Left keys [1]: [ws_sold_date_sk#72]
Right keys [1]: [d_date_sk#6]
Join condition: None
(59) Project [codegen id : 18]
(59) Project [codegen id : 17]
Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71]
Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6]
@ -376,10 +376,10 @@ Location [not included in comparison]/{warehouse_dir}/web_page]
PushedFilters: [IsNotNull(wp_web_page_sk)]
ReadSchema: struct<wp_web_page_sk:int>
(61) ColumnarToRow [codegen id : 17]
(61) ColumnarToRow [codegen id : 16]
Input [1]: [wp_web_page_sk#73]
(62) Filter [codegen id : 17]
(62) Filter [codegen id : 16]
Input [1]: [wp_web_page_sk#73]
Condition : isnotnull(wp_web_page_sk#73)
@ -387,16 +387,16 @@ Condition : isnotnull(wp_web_page_sk#73)
Input [1]: [wp_web_page_sk#73]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74]
(64) BroadcastHashJoin [codegen id : 18]
(64) BroadcastHashJoin [codegen id : 17]
Left keys [1]: [ws_web_page_sk#69]
Right keys [1]: [wp_web_page_sk#73]
Join condition: None
(65) Project [codegen id : 18]
(65) Project [codegen id : 17]
Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
(66) HashAggregate [codegen id : 18]
(66) HashAggregate [codegen id : 17]
Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
Keys [1]: [wp_web_page_sk#73]
Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))]
@ -407,7 +407,7 @@ Results [3]: [wp_web_page_sk#73, sum#77, sum#78]
Input [3]: [wp_web_page_sk#73, sum#77, sum#78]
Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79]
(68) HashAggregate [codegen id : 23]
(68) HashAggregate [codegen id : 22]
Input [3]: [wp_web_page_sk#73, sum#77, sum#78]
Keys [1]: [wp_web_page_sk#73]
Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))]
@ -422,38 +422,38 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(w
PushedFilters: [IsNotNull(wr_web_page_sk)]
ReadSchema: struct<wr_web_page_sk:bigint,wr_return_amt:decimal(7,2),wr_net_loss:decimal(7,2)>
(70) ColumnarToRow [codegen id : 21]
(70) ColumnarToRow [codegen id : 20]
Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87]
(71) Filter [codegen id : 21]
(71) Filter [codegen id : 20]
Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87]
Condition : isnotnull(wr_web_page_sk#84)
(72) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#6]
(73) BroadcastHashJoin [codegen id : 21]
(73) BroadcastHashJoin [codegen id : 20]
Left keys [1]: [wr_returned_date_sk#87]
Right keys [1]: [cast(d_date_sk#6 as bigint)]
Join condition: None
(74) Project [codegen id : 21]
(74) Project [codegen id : 20]
Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86]
Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6]
(75) ReusedExchange [Reuses operator id: 63]
Output [1]: [wp_web_page_sk#88]
(76) BroadcastHashJoin [codegen id : 21]
(76) BroadcastHashJoin [codegen id : 20]
Left keys [1]: [wr_web_page_sk#84]
Right keys [1]: [cast(wp_web_page_sk#88 as bigint)]
Join condition: None
(77) Project [codegen id : 21]
(77) Project [codegen id : 20]
Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88]
Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88]
(78) HashAggregate [codegen id : 21]
(78) HashAggregate [codegen id : 20]
Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88]
Keys [1]: [wp_web_page_sk#88]
Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))]
@ -464,7 +464,7 @@ Results [3]: [wp_web_page_sk#88, sum#91, sum#92]
Input [3]: [wp_web_page_sk#88, sum#91, sum#92]
Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93]
(80) HashAggregate [codegen id : 22]
(80) HashAggregate [codegen id : 21]
Input [3]: [wp_web_page_sk#88, sum#91, sum#92]
Keys [1]: [wp_web_page_sk#88]
Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))]
@ -475,22 +475,22 @@ Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85)
Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98]
(82) BroadcastHashJoin [codegen id : 23]
(82) BroadcastHashJoin [codegen id : 22]
Left keys [1]: [wp_web_page_sk#73]
Right keys [1]: [wp_web_page_sk#88]
Join condition: None
(83) Project [codegen id : 23]
(83) Project [codegen id : 22]
Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102]
Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97]
(84) Union
(85) Expand [codegen id : 24]
(85) Expand [codegen id : 23]
Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39]
Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105]
(86) HashAggregate [codegen id : 24]
(86) HashAggregate [codegen id : 23]
Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105]
Keys [3]: [channel#103, id#104, spark_grouping_id#105]
Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)]
@ -501,7 +501,7 @@ Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113,
Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117]
Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118]
(88) HashAggregate [codegen id : 25]
(88) HashAggregate [codegen id : 24]
Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117]
Keys [3]: [channel#103, id#104, spark_grouping_id#105]
Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)]

View file

@ -1,9 +1,9 @@
TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (25)
WholeStageCodegen (24)
HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
Exchange [channel,id,spark_grouping_id] #1
WholeStageCodegen (24)
WholeStageCodegen (23)
HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
Expand [sales,returns,profit,channel,id]
InputAdapter
@ -63,10 +63,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedExchange [d_date_sk] #3
InputAdapter
ReusedExchange [s_store_sk] #4
WholeStageCodegen (15)
WholeStageCodegen (14)
Project [sales,returns,profit,profit_loss,cs_call_center_sk]
InputAdapter
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
InputAdapter
BroadcastExchange #7
WholeStageCodegen (11)
HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum]
@ -82,27 +82,26 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #3
WholeStageCodegen (14)
HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum]
InputAdapter
Exchange #9
WholeStageCodegen (13)
HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum]
Project [cr_return_amount,cr_net_loss]
BroadcastHashJoin [cr_returned_date_sk,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #3
WholeStageCodegen (23)
HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum]
InputAdapter
Exchange #9
WholeStageCodegen (13)
HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum]
Project [cr_return_amount,cr_net_loss]
BroadcastHashJoin [cr_returned_date_sk,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #3
WholeStageCodegen (22)
Project [sales,returns,profit,profit_loss,wp_web_page_sk]
BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk]
HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum]
InputAdapter
Exchange [wp_web_page_sk] #10
WholeStageCodegen (18)
WholeStageCodegen (17)
HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum]
Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
@ -117,18 +116,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedExchange [d_date_sk] #3
InputAdapter
BroadcastExchange #11
WholeStageCodegen (17)
WholeStageCodegen (16)
Filter [wp_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_page [wp_web_page_sk]
InputAdapter
BroadcastExchange #12
WholeStageCodegen (22)
WholeStageCodegen (21)
HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum]
InputAdapter
Exchange [wp_web_page_sk] #13
WholeStageCodegen (21)
WholeStageCodegen (20)
HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum]
Project [wr_return_amt,wr_net_loss,wp_web_page_sk]
BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk]

View file

@ -1,11 +1,11 @@
== Physical Plan ==
BroadcastNestedLoopJoin Inner BuildRight (182)
:- BroadcastNestedLoopJoin Inner BuildRight (160)
: :- BroadcastNestedLoopJoin Inner BuildRight (138)
: : :- BroadcastNestedLoopJoin Inner BuildRight (116)
: : : :- BroadcastNestedLoopJoin Inner BuildRight (94)
: : : : :- BroadcastNestedLoopJoin Inner BuildRight (72)
: : : : : :- BroadcastNestedLoopJoin Inner BuildRight (50)
* BroadcastNestedLoopJoin Inner BuildRight (182)
:- * BroadcastNestedLoopJoin Inner BuildRight (160)
: :- * BroadcastNestedLoopJoin Inner BuildRight (138)
: : :- * BroadcastNestedLoopJoin Inner BuildRight (116)
: : : :- * BroadcastNestedLoopJoin Inner BuildRight (94)
: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (72)
: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (50)
: : : : : : :- * HashAggregate (28)
: : : : : : : +- Exchange (27)
: : : : : : : +- * HashAggregate (26)
@ -305,7 +305,7 @@ Results [1]: [count#17]
Input [1]: [count#17]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#18]
(28) HashAggregate [codegen id : 5]
(28) HashAggregate [codegen id : 40]
Input [1]: [count#17]
Keys: []
Functions [1]: [count(1)]
@ -319,14 +319,14 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(30) ColumnarToRow [codegen id : 9]
(30) ColumnarToRow [codegen id : 8]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(31) Filter [codegen id : 9]
(31) Filter [codegen id : 8]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(32) Project [codegen id : 9]
(32) Project [codegen id : 8]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
@ -337,14 +337,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(34) ColumnarToRow [codegen id : 6]
(34) ColumnarToRow [codegen id : 5]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
(35) Filter [codegen id : 6]
(35) Filter [codegen id : 5]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5))
(36) Project [codegen id : 6]
(36) Project [codegen id : 5]
Output [1]: [t_time_sk#5]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
@ -352,40 +352,40 @@ Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Input [1]: [t_time_sk#5]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21]
(38) BroadcastHashJoin [codegen id : 9]
(38) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#5]
Join condition: None
(39) Project [codegen id : 9]
(39) Project [codegen id : 8]
Output [2]: [ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5]
(40) ReusedExchange [Reuses operator id: 16]
Output [1]: [s_store_sk#9]
(41) BroadcastHashJoin [codegen id : 9]
(41) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#9]
Join condition: None
(42) Project [codegen id : 9]
(42) Project [codegen id : 8]
Output [1]: [ss_hdemo_sk#2]
Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9]
(43) ReusedExchange [Reuses operator id: 23]
Output [1]: [hd_demo_sk#12]
(44) BroadcastHashJoin [codegen id : 9]
(44) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#12]
Join condition: None
(45) Project [codegen id : 9]
(45) Project [codegen id : 8]
Output: []
Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12]
(46) HashAggregate [codegen id : 9]
(46) HashAggregate [codegen id : 8]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -396,7 +396,7 @@ Results [1]: [count#23]
Input [1]: [count#23]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24]
(48) HashAggregate [codegen id : 10]
(48) HashAggregate [codegen id : 9]
Input [1]: [count#23]
Keys: []
Functions [1]: [count(1)]
@ -407,7 +407,7 @@ Results [1]: [count(1)#25 AS h9_to_9_30#26]
Input [1]: [h9_to_9_30#26]
Arguments: IdentityBroadcastMode, [id=#27]
(50) BroadcastNestedLoopJoin
(50) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(51) Scan parquet default.store_sales
@ -417,14 +417,14 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(52) ColumnarToRow [codegen id : 14]
(52) ColumnarToRow [codegen id : 13]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(53) Filter [codegen id : 14]
(53) Filter [codegen id : 13]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(54) Project [codegen id : 14]
(54) Project [codegen id : 13]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
@ -435,14 +435,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(56) ColumnarToRow [codegen id : 11]
(56) ColumnarToRow [codegen id : 10]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
(57) Filter [codegen id : 11]
(57) Filter [codegen id : 10]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5))
(58) Project [codegen id : 11]
(58) Project [codegen id : 10]
Output [1]: [t_time_sk#5]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
@ -450,40 +450,40 @@ Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Input [1]: [t_time_sk#5]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28]
(60) BroadcastHashJoin [codegen id : 14]
(60) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#5]
Join condition: None
(61) Project [codegen id : 14]
(61) Project [codegen id : 13]
Output [2]: [ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5]
(62) ReusedExchange [Reuses operator id: 16]
Output [1]: [s_store_sk#9]
(63) BroadcastHashJoin [codegen id : 14]
(63) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#9]
Join condition: None
(64) Project [codegen id : 14]
(64) Project [codegen id : 13]
Output [1]: [ss_hdemo_sk#2]
Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9]
(65) ReusedExchange [Reuses operator id: 23]
Output [1]: [hd_demo_sk#12]
(66) BroadcastHashJoin [codegen id : 14]
(66) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#12]
Join condition: None
(67) Project [codegen id : 14]
(67) Project [codegen id : 13]
Output: []
Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12]
(68) HashAggregate [codegen id : 14]
(68) HashAggregate [codegen id : 13]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -494,7 +494,7 @@ Results [1]: [count#30]
Input [1]: [count#30]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31]
(70) HashAggregate [codegen id : 15]
(70) HashAggregate [codegen id : 14]
Input [1]: [count#30]
Keys: []
Functions [1]: [count(1)]
@ -505,7 +505,7 @@ Results [1]: [count(1)#32 AS h9_30_to_10#33]
Input [1]: [h9_30_to_10#33]
Arguments: IdentityBroadcastMode, [id=#34]
(72) BroadcastNestedLoopJoin
(72) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(73) Scan parquet default.store_sales
@ -515,14 +515,14 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(74) ColumnarToRow [codegen id : 19]
(74) ColumnarToRow [codegen id : 18]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(75) Filter [codegen id : 19]
(75) Filter [codegen id : 18]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(76) Project [codegen id : 19]
(76) Project [codegen id : 18]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
@ -533,14 +533,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(78) ColumnarToRow [codegen id : 16]
(78) ColumnarToRow [codegen id : 15]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
(79) Filter [codegen id : 16]
(79) Filter [codegen id : 15]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5))
(80) Project [codegen id : 16]
(80) Project [codegen id : 15]
Output [1]: [t_time_sk#5]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
@ -548,40 +548,40 @@ Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Input [1]: [t_time_sk#5]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35]
(82) BroadcastHashJoin [codegen id : 19]
(82) BroadcastHashJoin [codegen id : 18]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#5]
Join condition: None
(83) Project [codegen id : 19]
(83) Project [codegen id : 18]
Output [2]: [ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5]
(84) ReusedExchange [Reuses operator id: 16]
Output [1]: [s_store_sk#9]
(85) BroadcastHashJoin [codegen id : 19]
(85) BroadcastHashJoin [codegen id : 18]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#9]
Join condition: None
(86) Project [codegen id : 19]
(86) Project [codegen id : 18]
Output [1]: [ss_hdemo_sk#2]
Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9]
(87) ReusedExchange [Reuses operator id: 23]
Output [1]: [hd_demo_sk#12]
(88) BroadcastHashJoin [codegen id : 19]
(88) BroadcastHashJoin [codegen id : 18]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#12]
Join condition: None
(89) Project [codegen id : 19]
(89) Project [codegen id : 18]
Output: []
Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12]
(90) HashAggregate [codegen id : 19]
(90) HashAggregate [codegen id : 18]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -592,7 +592,7 @@ Results [1]: [count#37]
Input [1]: [count#37]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38]
(92) HashAggregate [codegen id : 20]
(92) HashAggregate [codegen id : 19]
Input [1]: [count#37]
Keys: []
Functions [1]: [count(1)]
@ -603,7 +603,7 @@ Results [1]: [count(1)#39 AS h10_to_10_30#40]
Input [1]: [h10_to_10_30#40]
Arguments: IdentityBroadcastMode, [id=#41]
(94) BroadcastNestedLoopJoin
(94) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(95) Scan parquet default.store_sales
@ -613,14 +613,14 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(96) ColumnarToRow [codegen id : 24]
(96) ColumnarToRow [codegen id : 23]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(97) Filter [codegen id : 24]
(97) Filter [codegen id : 23]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(98) Project [codegen id : 24]
(98) Project [codegen id : 23]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
@ -631,14 +631,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(100) ColumnarToRow [codegen id : 21]
(100) ColumnarToRow [codegen id : 20]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
(101) Filter [codegen id : 21]
(101) Filter [codegen id : 20]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5))
(102) Project [codegen id : 21]
(102) Project [codegen id : 20]
Output [1]: [t_time_sk#5]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
@ -646,40 +646,40 @@ Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Input [1]: [t_time_sk#5]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42]
(104) BroadcastHashJoin [codegen id : 24]
(104) BroadcastHashJoin [codegen id : 23]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#5]
Join condition: None
(105) Project [codegen id : 24]
(105) Project [codegen id : 23]
Output [2]: [ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5]
(106) ReusedExchange [Reuses operator id: 16]
Output [1]: [s_store_sk#9]
(107) BroadcastHashJoin [codegen id : 24]
(107) BroadcastHashJoin [codegen id : 23]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#9]
Join condition: None
(108) Project [codegen id : 24]
(108) Project [codegen id : 23]
Output [1]: [ss_hdemo_sk#2]
Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9]
(109) ReusedExchange [Reuses operator id: 23]
Output [1]: [hd_demo_sk#12]
(110) BroadcastHashJoin [codegen id : 24]
(110) BroadcastHashJoin [codegen id : 23]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#12]
Join condition: None
(111) Project [codegen id : 24]
(111) Project [codegen id : 23]
Output: []
Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12]
(112) HashAggregate [codegen id : 24]
(112) HashAggregate [codegen id : 23]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -690,7 +690,7 @@ Results [1]: [count#44]
Input [1]: [count#44]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45]
(114) HashAggregate [codegen id : 25]
(114) HashAggregate [codegen id : 24]
Input [1]: [count#44]
Keys: []
Functions [1]: [count(1)]
@ -701,7 +701,7 @@ Results [1]: [count(1)#46 AS h10_30_to_11#47]
Input [1]: [h10_30_to_11#47]
Arguments: IdentityBroadcastMode, [id=#48]
(116) BroadcastNestedLoopJoin
(116) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(117) Scan parquet default.store_sales
@ -711,14 +711,14 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(118) ColumnarToRow [codegen id : 29]
(118) ColumnarToRow [codegen id : 28]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(119) Filter [codegen id : 29]
(119) Filter [codegen id : 28]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(120) Project [codegen id : 29]
(120) Project [codegen id : 28]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
@ -729,14 +729,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(122) ColumnarToRow [codegen id : 26]
(122) ColumnarToRow [codegen id : 25]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
(123) Filter [codegen id : 26]
(123) Filter [codegen id : 25]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5))
(124) Project [codegen id : 26]
(124) Project [codegen id : 25]
Output [1]: [t_time_sk#5]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
@ -744,40 +744,40 @@ Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Input [1]: [t_time_sk#5]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49]
(126) BroadcastHashJoin [codegen id : 29]
(126) BroadcastHashJoin [codegen id : 28]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#5]
Join condition: None
(127) Project [codegen id : 29]
(127) Project [codegen id : 28]
Output [2]: [ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5]
(128) ReusedExchange [Reuses operator id: 16]
Output [1]: [s_store_sk#9]
(129) BroadcastHashJoin [codegen id : 29]
(129) BroadcastHashJoin [codegen id : 28]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#9]
Join condition: None
(130) Project [codegen id : 29]
(130) Project [codegen id : 28]
Output [1]: [ss_hdemo_sk#2]
Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9]
(131) ReusedExchange [Reuses operator id: 23]
Output [1]: [hd_demo_sk#12]
(132) BroadcastHashJoin [codegen id : 29]
(132) BroadcastHashJoin [codegen id : 28]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#12]
Join condition: None
(133) Project [codegen id : 29]
(133) Project [codegen id : 28]
Output: []
Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12]
(134) HashAggregate [codegen id : 29]
(134) HashAggregate [codegen id : 28]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -788,7 +788,7 @@ Results [1]: [count#51]
Input [1]: [count#51]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52]
(136) HashAggregate [codegen id : 30]
(136) HashAggregate [codegen id : 29]
Input [1]: [count#51]
Keys: []
Functions [1]: [count(1)]
@ -799,7 +799,7 @@ Results [1]: [count(1)#53 AS h11_to_11_30#54]
Input [1]: [h11_to_11_30#54]
Arguments: IdentityBroadcastMode, [id=#55]
(138) BroadcastNestedLoopJoin
(138) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(139) Scan parquet default.store_sales
@ -809,14 +809,14 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(140) ColumnarToRow [codegen id : 34]
(140) ColumnarToRow [codegen id : 33]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(141) Filter [codegen id : 34]
(141) Filter [codegen id : 33]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(142) Project [codegen id : 34]
(142) Project [codegen id : 33]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
@ -827,14 +827,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(144) ColumnarToRow [codegen id : 31]
(144) ColumnarToRow [codegen id : 30]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
(145) Filter [codegen id : 31]
(145) Filter [codegen id : 30]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5))
(146) Project [codegen id : 31]
(146) Project [codegen id : 30]
Output [1]: [t_time_sk#5]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
@ -842,40 +842,40 @@ Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Input [1]: [t_time_sk#5]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56]
(148) BroadcastHashJoin [codegen id : 34]
(148) BroadcastHashJoin [codegen id : 33]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#5]
Join condition: None
(149) Project [codegen id : 34]
(149) Project [codegen id : 33]
Output [2]: [ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5]
(150) ReusedExchange [Reuses operator id: 16]
Output [1]: [s_store_sk#9]
(151) BroadcastHashJoin [codegen id : 34]
(151) BroadcastHashJoin [codegen id : 33]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#9]
Join condition: None
(152) Project [codegen id : 34]
(152) Project [codegen id : 33]
Output [1]: [ss_hdemo_sk#2]
Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9]
(153) ReusedExchange [Reuses operator id: 23]
Output [1]: [hd_demo_sk#12]
(154) BroadcastHashJoin [codegen id : 34]
(154) BroadcastHashJoin [codegen id : 33]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#12]
Join condition: None
(155) Project [codegen id : 34]
(155) Project [codegen id : 33]
Output: []
Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12]
(156) HashAggregate [codegen id : 34]
(156) HashAggregate [codegen id : 33]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -886,7 +886,7 @@ Results [1]: [count#58]
Input [1]: [count#58]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59]
(158) HashAggregate [codegen id : 35]
(158) HashAggregate [codegen id : 34]
Input [1]: [count#58]
Keys: []
Functions [1]: [count(1)]
@ -897,7 +897,7 @@ Results [1]: [count(1)#60 AS h11_30_to_12#61]
Input [1]: [h11_30_to_12#61]
Arguments: IdentityBroadcastMode, [id=#62]
(160) BroadcastNestedLoopJoin
(160) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(161) Scan parquet default.store_sales
@ -907,14 +907,14 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(162) ColumnarToRow [codegen id : 39]
(162) ColumnarToRow [codegen id : 38]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(163) Filter [codegen id : 39]
(163) Filter [codegen id : 38]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(164) Project [codegen id : 39]
(164) Project [codegen id : 38]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
@ -925,14 +925,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(166) ColumnarToRow [codegen id : 36]
(166) ColumnarToRow [codegen id : 35]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
(167) Filter [codegen id : 36]
(167) Filter [codegen id : 35]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 12)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5))
(168) Project [codegen id : 36]
(168) Project [codegen id : 35]
Output [1]: [t_time_sk#5]
Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
@ -940,40 +940,40 @@ Input [3]: [t_time_sk#5, t_hour#6, t_minute#7]
Input [1]: [t_time_sk#5]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63]
(170) BroadcastHashJoin [codegen id : 39]
(170) BroadcastHashJoin [codegen id : 38]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#5]
Join condition: None
(171) Project [codegen id : 39]
(171) Project [codegen id : 38]
Output [2]: [ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5]
(172) ReusedExchange [Reuses operator id: 16]
Output [1]: [s_store_sk#9]
(173) BroadcastHashJoin [codegen id : 39]
(173) BroadcastHashJoin [codegen id : 38]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#9]
Join condition: None
(174) Project [codegen id : 39]
(174) Project [codegen id : 38]
Output [1]: [ss_hdemo_sk#2]
Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9]
(175) ReusedExchange [Reuses operator id: 23]
Output [1]: [hd_demo_sk#12]
(176) BroadcastHashJoin [codegen id : 39]
(176) BroadcastHashJoin [codegen id : 38]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#12]
Join condition: None
(177) Project [codegen id : 39]
(177) Project [codegen id : 38]
Output: []
Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12]
(178) HashAggregate [codegen id : 39]
(178) HashAggregate [codegen id : 38]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -984,7 +984,7 @@ Results [1]: [count#65]
Input [1]: [count#65]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66]
(180) HashAggregate [codegen id : 40]
(180) HashAggregate [codegen id : 39]
Input [1]: [count#65]
Keys: []
Functions [1]: [count(1)]
@ -995,6 +995,6 @@ Results [1]: [count(1)#67 AS h12_to_12_30#68]
Input [1]: [h12_to_12_30#68]
Arguments: IdentityBroadcastMode, [id=#69]
(182) BroadcastNestedLoopJoin
(182) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None

View file

@ -1,11 +1,11 @@
BroadcastNestedLoopJoin
WholeStageCodegen (40)
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
WholeStageCodegen (5)
BroadcastNestedLoopJoin
HashAggregate [count] [count(1),h8_30_to_9,count]
InputAdapter
Exchange #1
@ -46,12 +46,75 @@ BroadcastNestedLoopJoin
ColumnarToRow
InputAdapter
Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
BroadcastExchange #5
WholeStageCodegen (10)
HashAggregate [count] [count(1),h9_to_9_30,count]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (9)
HashAggregate [count] [count(1),h9_to_9_30,count]
InputAdapter
Exchange #6
WholeStageCodegen (8)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
Project [ss_hdemo_sk]
BroadcastHashJoin [ss_store_sk,s_store_sk]
Project [ss_hdemo_sk,ss_store_sk]
BroadcastHashJoin [ss_sold_time_sk,t_time_sk]
Project [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
BroadcastExchange #7
WholeStageCodegen (5)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #3
InputAdapter
ReusedExchange [hd_demo_sk] #4
InputAdapter
BroadcastExchange #8
WholeStageCodegen (14)
HashAggregate [count] [count(1),h9_30_to_10,count]
InputAdapter
Exchange #9
WholeStageCodegen (13)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
Project [ss_hdemo_sk]
BroadcastHashJoin [ss_store_sk,s_store_sk]
Project [ss_hdemo_sk,ss_store_sk]
BroadcastHashJoin [ss_sold_time_sk,t_time_sk]
Project [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
BroadcastExchange #10
WholeStageCodegen (10)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #3
InputAdapter
ReusedExchange [hd_demo_sk] #4
InputAdapter
BroadcastExchange #11
WholeStageCodegen (19)
HashAggregate [count] [count(1),h10_to_10_30,count]
InputAdapter
Exchange #6
WholeStageCodegen (9)
Exchange #12
WholeStageCodegen (18)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
@ -65,8 +128,8 @@ BroadcastNestedLoopJoin
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
BroadcastExchange #7
WholeStageCodegen (6)
BroadcastExchange #13
WholeStageCodegen (15)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
@ -76,12 +139,13 @@ BroadcastNestedLoopJoin
ReusedExchange [s_store_sk] #3
InputAdapter
ReusedExchange [hd_demo_sk] #4
BroadcastExchange #8
WholeStageCodegen (15)
HashAggregate [count] [count(1),h9_30_to_10,count]
InputAdapter
BroadcastExchange #14
WholeStageCodegen (24)
HashAggregate [count] [count(1),h10_30_to_11,count]
InputAdapter
Exchange #9
WholeStageCodegen (14)
Exchange #15
WholeStageCodegen (23)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
@ -95,8 +159,8 @@ BroadcastNestedLoopJoin
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
BroadcastExchange #10
WholeStageCodegen (11)
BroadcastExchange #16
WholeStageCodegen (20)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
@ -106,12 +170,13 @@ BroadcastNestedLoopJoin
ReusedExchange [s_store_sk] #3
InputAdapter
ReusedExchange [hd_demo_sk] #4
BroadcastExchange #11
WholeStageCodegen (20)
HashAggregate [count] [count(1),h10_to_10_30,count]
InputAdapter
BroadcastExchange #17
WholeStageCodegen (29)
HashAggregate [count] [count(1),h11_to_11_30,count]
InputAdapter
Exchange #12
WholeStageCodegen (19)
Exchange #18
WholeStageCodegen (28)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
@ -125,8 +190,8 @@ BroadcastNestedLoopJoin
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
BroadcastExchange #13
WholeStageCodegen (16)
BroadcastExchange #19
WholeStageCodegen (25)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
@ -136,12 +201,13 @@ BroadcastNestedLoopJoin
ReusedExchange [s_store_sk] #3
InputAdapter
ReusedExchange [hd_demo_sk] #4
BroadcastExchange #14
WholeStageCodegen (25)
HashAggregate [count] [count(1),h10_30_to_11,count]
InputAdapter
BroadcastExchange #20
WholeStageCodegen (34)
HashAggregate [count] [count(1),h11_30_to_12,count]
InputAdapter
Exchange #15
WholeStageCodegen (24)
Exchange #21
WholeStageCodegen (33)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
@ -155,8 +221,8 @@ BroadcastNestedLoopJoin
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
BroadcastExchange #16
WholeStageCodegen (21)
BroadcastExchange #22
WholeStageCodegen (30)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
@ -166,12 +232,13 @@ BroadcastNestedLoopJoin
ReusedExchange [s_store_sk] #3
InputAdapter
ReusedExchange [hd_demo_sk] #4
BroadcastExchange #17
WholeStageCodegen (30)
HashAggregate [count] [count(1),h11_to_11_30,count]
InputAdapter
BroadcastExchange #23
WholeStageCodegen (39)
HashAggregate [count] [count(1),h12_to_12_30,count]
InputAdapter
Exchange #18
WholeStageCodegen (29)
Exchange #24
WholeStageCodegen (38)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
@ -185,8 +252,8 @@ BroadcastNestedLoopJoin
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
BroadcastExchange #19
WholeStageCodegen (26)
BroadcastExchange #25
WholeStageCodegen (35)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
@ -196,63 +263,3 @@ BroadcastNestedLoopJoin
ReusedExchange [s_store_sk] #3
InputAdapter
ReusedExchange [hd_demo_sk] #4
BroadcastExchange #20
WholeStageCodegen (35)
HashAggregate [count] [count(1),h11_30_to_12,count]
InputAdapter
Exchange #21
WholeStageCodegen (34)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
Project [ss_hdemo_sk]
BroadcastHashJoin [ss_store_sk,s_store_sk]
Project [ss_hdemo_sk,ss_store_sk]
BroadcastHashJoin [ss_sold_time_sk,t_time_sk]
Project [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
BroadcastExchange #22
WholeStageCodegen (31)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #3
InputAdapter
ReusedExchange [hd_demo_sk] #4
BroadcastExchange #23
WholeStageCodegen (40)
HashAggregate [count] [count(1),h12_to_12_30,count]
InputAdapter
Exchange #24
WholeStageCodegen (39)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
Project [ss_hdemo_sk]
BroadcastHashJoin [ss_store_sk,s_store_sk]
Project [ss_hdemo_sk,ss_store_sk]
BroadcastHashJoin [ss_sold_time_sk,t_time_sk]
Project [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
BroadcastExchange #25
WholeStageCodegen (36)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #3
InputAdapter
ReusedExchange [hd_demo_sk] #4

View file

@ -1,11 +1,11 @@
== Physical Plan ==
BroadcastNestedLoopJoin Inner BuildRight (182)
:- BroadcastNestedLoopJoin Inner BuildRight (160)
: :- BroadcastNestedLoopJoin Inner BuildRight (138)
: : :- BroadcastNestedLoopJoin Inner BuildRight (116)
: : : :- BroadcastNestedLoopJoin Inner BuildRight (94)
: : : : :- BroadcastNestedLoopJoin Inner BuildRight (72)
: : : : : :- BroadcastNestedLoopJoin Inner BuildRight (50)
* BroadcastNestedLoopJoin Inner BuildRight (182)
:- * BroadcastNestedLoopJoin Inner BuildRight (160)
: :- * BroadcastNestedLoopJoin Inner BuildRight (138)
: : :- * BroadcastNestedLoopJoin Inner BuildRight (116)
: : : :- * BroadcastNestedLoopJoin Inner BuildRight (94)
: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (72)
: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (50)
: : : : : : :- * HashAggregate (28)
: : : : : : : +- Exchange (27)
: : : : : : : +- * HashAggregate (26)
@ -305,7 +305,7 @@ Results [1]: [count#17]
Input [1]: [count#17]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#18]
(28) HashAggregate [codegen id : 5]
(28) HashAggregate [codegen id : 40]
Input [1]: [count#17]
Keys: []
Functions [1]: [count(1)]
@ -319,26 +319,26 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(30) ColumnarToRow [codegen id : 9]
(30) ColumnarToRow [codegen id : 8]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(31) Filter [codegen id : 9]
(31) Filter [codegen id : 8]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(32) Project [codegen id : 9]
(32) Project [codegen id : 8]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(33) ReusedExchange [Reuses operator id: 9]
Output [1]: [hd_demo_sk#5]
(34) BroadcastHashJoin [codegen id : 9]
(34) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#5]
Join condition: None
(35) Project [codegen id : 9]
(35) Project [codegen id : 8]
Output [2]: [ss_sold_time_sk#1, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5]
@ -349,14 +349,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(37) ColumnarToRow [codegen id : 7]
(37) ColumnarToRow [codegen id : 6]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
(38) Filter [codegen id : 7]
(38) Filter [codegen id : 6]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9))
(39) Project [codegen id : 7]
(39) Project [codegen id : 6]
Output [1]: [t_time_sk#9]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
@ -364,28 +364,28 @@ Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Input [1]: [t_time_sk#9]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21]
(41) BroadcastHashJoin [codegen id : 9]
(41) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#9]
Join condition: None
(42) Project [codegen id : 9]
(42) Project [codegen id : 8]
Output [1]: [ss_store_sk#3]
Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9]
(43) ReusedExchange [Reuses operator id: 23]
Output [1]: [s_store_sk#13]
(44) BroadcastHashJoin [codegen id : 9]
(44) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#13]
Join condition: None
(45) Project [codegen id : 9]
(45) Project [codegen id : 8]
Output: []
Input [2]: [ss_store_sk#3, s_store_sk#13]
(46) HashAggregate [codegen id : 9]
(46) HashAggregate [codegen id : 8]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -396,7 +396,7 @@ Results [1]: [count#23]
Input [1]: [count#23]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24]
(48) HashAggregate [codegen id : 10]
(48) HashAggregate [codegen id : 9]
Input [1]: [count#23]
Keys: []
Functions [1]: [count(1)]
@ -407,7 +407,7 @@ Results [1]: [count(1)#25 AS h9_to_9_30#26]
Input [1]: [h9_to_9_30#26]
Arguments: IdentityBroadcastMode, [id=#27]
(50) BroadcastNestedLoopJoin
(50) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(51) Scan parquet default.store_sales
@ -417,26 +417,26 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(52) ColumnarToRow [codegen id : 14]
(52) ColumnarToRow [codegen id : 13]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(53) Filter [codegen id : 14]
(53) Filter [codegen id : 13]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(54) Project [codegen id : 14]
(54) Project [codegen id : 13]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(55) ReusedExchange [Reuses operator id: 9]
Output [1]: [hd_demo_sk#5]
(56) BroadcastHashJoin [codegen id : 14]
(56) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#5]
Join condition: None
(57) Project [codegen id : 14]
(57) Project [codegen id : 13]
Output [2]: [ss_sold_time_sk#1, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5]
@ -447,14 +447,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(59) ColumnarToRow [codegen id : 12]
(59) ColumnarToRow [codegen id : 11]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
(60) Filter [codegen id : 12]
(60) Filter [codegen id : 11]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9))
(61) Project [codegen id : 12]
(61) Project [codegen id : 11]
Output [1]: [t_time_sk#9]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
@ -462,28 +462,28 @@ Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Input [1]: [t_time_sk#9]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28]
(63) BroadcastHashJoin [codegen id : 14]
(63) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#9]
Join condition: None
(64) Project [codegen id : 14]
(64) Project [codegen id : 13]
Output [1]: [ss_store_sk#3]
Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9]
(65) ReusedExchange [Reuses operator id: 23]
Output [1]: [s_store_sk#13]
(66) BroadcastHashJoin [codegen id : 14]
(66) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#13]
Join condition: None
(67) Project [codegen id : 14]
(67) Project [codegen id : 13]
Output: []
Input [2]: [ss_store_sk#3, s_store_sk#13]
(68) HashAggregate [codegen id : 14]
(68) HashAggregate [codegen id : 13]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -494,7 +494,7 @@ Results [1]: [count#30]
Input [1]: [count#30]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31]
(70) HashAggregate [codegen id : 15]
(70) HashAggregate [codegen id : 14]
Input [1]: [count#30]
Keys: []
Functions [1]: [count(1)]
@ -505,7 +505,7 @@ Results [1]: [count(1)#32 AS h9_30_to_10#33]
Input [1]: [h9_30_to_10#33]
Arguments: IdentityBroadcastMode, [id=#34]
(72) BroadcastNestedLoopJoin
(72) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(73) Scan parquet default.store_sales
@ -515,26 +515,26 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(74) ColumnarToRow [codegen id : 19]
(74) ColumnarToRow [codegen id : 18]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(75) Filter [codegen id : 19]
(75) Filter [codegen id : 18]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(76) Project [codegen id : 19]
(76) Project [codegen id : 18]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(77) ReusedExchange [Reuses operator id: 9]
Output [1]: [hd_demo_sk#5]
(78) BroadcastHashJoin [codegen id : 19]
(78) BroadcastHashJoin [codegen id : 18]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#5]
Join condition: None
(79) Project [codegen id : 19]
(79) Project [codegen id : 18]
Output [2]: [ss_sold_time_sk#1, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5]
@ -545,14 +545,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(81) ColumnarToRow [codegen id : 17]
(81) ColumnarToRow [codegen id : 16]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
(82) Filter [codegen id : 17]
(82) Filter [codegen id : 16]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9))
(83) Project [codegen id : 17]
(83) Project [codegen id : 16]
Output [1]: [t_time_sk#9]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
@ -560,28 +560,28 @@ Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Input [1]: [t_time_sk#9]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35]
(85) BroadcastHashJoin [codegen id : 19]
(85) BroadcastHashJoin [codegen id : 18]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#9]
Join condition: None
(86) Project [codegen id : 19]
(86) Project [codegen id : 18]
Output [1]: [ss_store_sk#3]
Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9]
(87) ReusedExchange [Reuses operator id: 23]
Output [1]: [s_store_sk#13]
(88) BroadcastHashJoin [codegen id : 19]
(88) BroadcastHashJoin [codegen id : 18]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#13]
Join condition: None
(89) Project [codegen id : 19]
(89) Project [codegen id : 18]
Output: []
Input [2]: [ss_store_sk#3, s_store_sk#13]
(90) HashAggregate [codegen id : 19]
(90) HashAggregate [codegen id : 18]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -592,7 +592,7 @@ Results [1]: [count#37]
Input [1]: [count#37]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38]
(92) HashAggregate [codegen id : 20]
(92) HashAggregate [codegen id : 19]
Input [1]: [count#37]
Keys: []
Functions [1]: [count(1)]
@ -603,7 +603,7 @@ Results [1]: [count(1)#39 AS h10_to_10_30#40]
Input [1]: [h10_to_10_30#40]
Arguments: IdentityBroadcastMode, [id=#41]
(94) BroadcastNestedLoopJoin
(94) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(95) Scan parquet default.store_sales
@ -613,26 +613,26 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(96) ColumnarToRow [codegen id : 24]
(96) ColumnarToRow [codegen id : 23]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(97) Filter [codegen id : 24]
(97) Filter [codegen id : 23]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(98) Project [codegen id : 24]
(98) Project [codegen id : 23]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(99) ReusedExchange [Reuses operator id: 9]
Output [1]: [hd_demo_sk#5]
(100) BroadcastHashJoin [codegen id : 24]
(100) BroadcastHashJoin [codegen id : 23]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#5]
Join condition: None
(101) Project [codegen id : 24]
(101) Project [codegen id : 23]
Output [2]: [ss_sold_time_sk#1, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5]
@ -643,14 +643,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(103) ColumnarToRow [codegen id : 22]
(103) ColumnarToRow [codegen id : 21]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
(104) Filter [codegen id : 22]
(104) Filter [codegen id : 21]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9))
(105) Project [codegen id : 22]
(105) Project [codegen id : 21]
Output [1]: [t_time_sk#9]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
@ -658,28 +658,28 @@ Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Input [1]: [t_time_sk#9]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42]
(107) BroadcastHashJoin [codegen id : 24]
(107) BroadcastHashJoin [codegen id : 23]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#9]
Join condition: None
(108) Project [codegen id : 24]
(108) Project [codegen id : 23]
Output [1]: [ss_store_sk#3]
Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9]
(109) ReusedExchange [Reuses operator id: 23]
Output [1]: [s_store_sk#13]
(110) BroadcastHashJoin [codegen id : 24]
(110) BroadcastHashJoin [codegen id : 23]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#13]
Join condition: None
(111) Project [codegen id : 24]
(111) Project [codegen id : 23]
Output: []
Input [2]: [ss_store_sk#3, s_store_sk#13]
(112) HashAggregate [codegen id : 24]
(112) HashAggregate [codegen id : 23]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -690,7 +690,7 @@ Results [1]: [count#44]
Input [1]: [count#44]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45]
(114) HashAggregate [codegen id : 25]
(114) HashAggregate [codegen id : 24]
Input [1]: [count#44]
Keys: []
Functions [1]: [count(1)]
@ -701,7 +701,7 @@ Results [1]: [count(1)#46 AS h10_30_to_11#47]
Input [1]: [h10_30_to_11#47]
Arguments: IdentityBroadcastMode, [id=#48]
(116) BroadcastNestedLoopJoin
(116) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(117) Scan parquet default.store_sales
@ -711,26 +711,26 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(118) ColumnarToRow [codegen id : 29]
(118) ColumnarToRow [codegen id : 28]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(119) Filter [codegen id : 29]
(119) Filter [codegen id : 28]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(120) Project [codegen id : 29]
(120) Project [codegen id : 28]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(121) ReusedExchange [Reuses operator id: 9]
Output [1]: [hd_demo_sk#5]
(122) BroadcastHashJoin [codegen id : 29]
(122) BroadcastHashJoin [codegen id : 28]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#5]
Join condition: None
(123) Project [codegen id : 29]
(123) Project [codegen id : 28]
Output [2]: [ss_sold_time_sk#1, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5]
@ -741,14 +741,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(125) ColumnarToRow [codegen id : 27]
(125) ColumnarToRow [codegen id : 26]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
(126) Filter [codegen id : 27]
(126) Filter [codegen id : 26]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9))
(127) Project [codegen id : 27]
(127) Project [codegen id : 26]
Output [1]: [t_time_sk#9]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
@ -756,28 +756,28 @@ Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Input [1]: [t_time_sk#9]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49]
(129) BroadcastHashJoin [codegen id : 29]
(129) BroadcastHashJoin [codegen id : 28]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#9]
Join condition: None
(130) Project [codegen id : 29]
(130) Project [codegen id : 28]
Output [1]: [ss_store_sk#3]
Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9]
(131) ReusedExchange [Reuses operator id: 23]
Output [1]: [s_store_sk#13]
(132) BroadcastHashJoin [codegen id : 29]
(132) BroadcastHashJoin [codegen id : 28]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#13]
Join condition: None
(133) Project [codegen id : 29]
(133) Project [codegen id : 28]
Output: []
Input [2]: [ss_store_sk#3, s_store_sk#13]
(134) HashAggregate [codegen id : 29]
(134) HashAggregate [codegen id : 28]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -788,7 +788,7 @@ Results [1]: [count#51]
Input [1]: [count#51]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52]
(136) HashAggregate [codegen id : 30]
(136) HashAggregate [codegen id : 29]
Input [1]: [count#51]
Keys: []
Functions [1]: [count(1)]
@ -799,7 +799,7 @@ Results [1]: [count(1)#53 AS h11_to_11_30#54]
Input [1]: [h11_to_11_30#54]
Arguments: IdentityBroadcastMode, [id=#55]
(138) BroadcastNestedLoopJoin
(138) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(139) Scan parquet default.store_sales
@ -809,26 +809,26 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(140) ColumnarToRow [codegen id : 34]
(140) ColumnarToRow [codegen id : 33]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(141) Filter [codegen id : 34]
(141) Filter [codegen id : 33]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(142) Project [codegen id : 34]
(142) Project [codegen id : 33]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(143) ReusedExchange [Reuses operator id: 9]
Output [1]: [hd_demo_sk#5]
(144) BroadcastHashJoin [codegen id : 34]
(144) BroadcastHashJoin [codegen id : 33]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#5]
Join condition: None
(145) Project [codegen id : 34]
(145) Project [codegen id : 33]
Output [2]: [ss_sold_time_sk#1, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5]
@ -839,14 +839,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(147) ColumnarToRow [codegen id : 32]
(147) ColumnarToRow [codegen id : 31]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
(148) Filter [codegen id : 32]
(148) Filter [codegen id : 31]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9))
(149) Project [codegen id : 32]
(149) Project [codegen id : 31]
Output [1]: [t_time_sk#9]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
@ -854,28 +854,28 @@ Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Input [1]: [t_time_sk#9]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56]
(151) BroadcastHashJoin [codegen id : 34]
(151) BroadcastHashJoin [codegen id : 33]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#9]
Join condition: None
(152) Project [codegen id : 34]
(152) Project [codegen id : 33]
Output [1]: [ss_store_sk#3]
Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9]
(153) ReusedExchange [Reuses operator id: 23]
Output [1]: [s_store_sk#13]
(154) BroadcastHashJoin [codegen id : 34]
(154) BroadcastHashJoin [codegen id : 33]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#13]
Join condition: None
(155) Project [codegen id : 34]
(155) Project [codegen id : 33]
Output: []
Input [2]: [ss_store_sk#3, s_store_sk#13]
(156) HashAggregate [codegen id : 34]
(156) HashAggregate [codegen id : 33]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -886,7 +886,7 @@ Results [1]: [count#58]
Input [1]: [count#58]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59]
(158) HashAggregate [codegen id : 35]
(158) HashAggregate [codegen id : 34]
Input [1]: [count#58]
Keys: []
Functions [1]: [count(1)]
@ -897,7 +897,7 @@ Results [1]: [count(1)#60 AS h11_30_to_12#61]
Input [1]: [h11_30_to_12#61]
Arguments: IdentityBroadcastMode, [id=#62]
(160) BroadcastNestedLoopJoin
(160) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None
(161) Scan parquet default.store_sales
@ -907,26 +907,26 @@ Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int>
(162) ColumnarToRow [codegen id : 39]
(162) ColumnarToRow [codegen id : 38]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(163) Filter [codegen id : 39]
(163) Filter [codegen id : 38]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3))
(164) Project [codegen id : 39]
(164) Project [codegen id : 38]
Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4]
(165) ReusedExchange [Reuses operator id: 9]
Output [1]: [hd_demo_sk#5]
(166) BroadcastHashJoin [codegen id : 39]
(166) BroadcastHashJoin [codegen id : 38]
Left keys [1]: [ss_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#5]
Join condition: None
(167) Project [codegen id : 39]
(167) Project [codegen id : 38]
Output [2]: [ss_sold_time_sk#1, ss_store_sk#3]
Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5]
@ -937,14 +937,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int>
(169) ColumnarToRow [codegen id : 37]
(169) ColumnarToRow [codegen id : 36]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
(170) Filter [codegen id : 37]
(170) Filter [codegen id : 36]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 12)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9))
(171) Project [codegen id : 37]
(171) Project [codegen id : 36]
Output [1]: [t_time_sk#9]
Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
@ -952,28 +952,28 @@ Input [3]: [t_time_sk#9, t_hour#10, t_minute#11]
Input [1]: [t_time_sk#9]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63]
(173) BroadcastHashJoin [codegen id : 39]
(173) BroadcastHashJoin [codegen id : 38]
Left keys [1]: [ss_sold_time_sk#1]
Right keys [1]: [t_time_sk#9]
Join condition: None
(174) Project [codegen id : 39]
(174) Project [codegen id : 38]
Output [1]: [ss_store_sk#3]
Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9]
(175) ReusedExchange [Reuses operator id: 23]
Output [1]: [s_store_sk#13]
(176) BroadcastHashJoin [codegen id : 39]
(176) BroadcastHashJoin [codegen id : 38]
Left keys [1]: [ss_store_sk#3]
Right keys [1]: [s_store_sk#13]
Join condition: None
(177) Project [codegen id : 39]
(177) Project [codegen id : 38]
Output: []
Input [2]: [ss_store_sk#3, s_store_sk#13]
(178) HashAggregate [codegen id : 39]
(178) HashAggregate [codegen id : 38]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -984,7 +984,7 @@ Results [1]: [count#65]
Input [1]: [count#65]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66]
(180) HashAggregate [codegen id : 40]
(180) HashAggregate [codegen id : 39]
Input [1]: [count#65]
Keys: []
Functions [1]: [count(1)]
@ -995,6 +995,6 @@ Results [1]: [count(1)#67 AS h12_to_12_30#68]
Input [1]: [h12_to_12_30#68]
Arguments: IdentityBroadcastMode, [id=#69]
(182) BroadcastNestedLoopJoin
(182) BroadcastNestedLoopJoin [codegen id : 40]
Join condition: None

View file

@ -1,11 +1,11 @@
BroadcastNestedLoopJoin
WholeStageCodegen (40)
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
WholeStageCodegen (5)
BroadcastNestedLoopJoin
HashAggregate [count] [count(1),h8_30_to_9,count]
InputAdapter
Exchange #1
@ -46,12 +46,75 @@ BroadcastNestedLoopJoin
ColumnarToRow
InputAdapter
Scan parquet default.store [s_store_sk,s_store_name]
BroadcastExchange #5
WholeStageCodegen (10)
HashAggregate [count] [count(1),h9_to_9_30,count]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (9)
HashAggregate [count] [count(1),h9_to_9_30,count]
InputAdapter
Exchange #6
WholeStageCodegen (8)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_store_sk,s_store_sk]
Project [ss_store_sk]
BroadcastHashJoin [ss_sold_time_sk,t_time_sk]
Project [ss_sold_time_sk,ss_store_sk]
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
Project [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
ReusedExchange [hd_demo_sk] #2
InputAdapter
BroadcastExchange #7
WholeStageCodegen (6)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #4
InputAdapter
BroadcastExchange #8
WholeStageCodegen (14)
HashAggregate [count] [count(1),h9_30_to_10,count]
InputAdapter
Exchange #9
WholeStageCodegen (13)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_store_sk,s_store_sk]
Project [ss_store_sk]
BroadcastHashJoin [ss_sold_time_sk,t_time_sk]
Project [ss_sold_time_sk,ss_store_sk]
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
Project [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
ReusedExchange [hd_demo_sk] #2
InputAdapter
BroadcastExchange #10
WholeStageCodegen (11)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #4
InputAdapter
BroadcastExchange #11
WholeStageCodegen (19)
HashAggregate [count] [count(1),h10_to_10_30,count]
InputAdapter
Exchange #6
WholeStageCodegen (9)
Exchange #12
WholeStageCodegen (18)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_store_sk,s_store_sk]
@ -67,8 +130,8 @@ BroadcastNestedLoopJoin
InputAdapter
ReusedExchange [hd_demo_sk] #2
InputAdapter
BroadcastExchange #7
WholeStageCodegen (7)
BroadcastExchange #13
WholeStageCodegen (16)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
@ -76,12 +139,13 @@ BroadcastNestedLoopJoin
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #4
BroadcastExchange #8
WholeStageCodegen (15)
HashAggregate [count] [count(1),h9_30_to_10,count]
InputAdapter
BroadcastExchange #14
WholeStageCodegen (24)
HashAggregate [count] [count(1),h10_30_to_11,count]
InputAdapter
Exchange #9
WholeStageCodegen (14)
Exchange #15
WholeStageCodegen (23)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_store_sk,s_store_sk]
@ -97,8 +161,8 @@ BroadcastNestedLoopJoin
InputAdapter
ReusedExchange [hd_demo_sk] #2
InputAdapter
BroadcastExchange #10
WholeStageCodegen (12)
BroadcastExchange #16
WholeStageCodegen (21)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
@ -106,12 +170,13 @@ BroadcastNestedLoopJoin
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #4
BroadcastExchange #11
WholeStageCodegen (20)
HashAggregate [count] [count(1),h10_to_10_30,count]
InputAdapter
BroadcastExchange #17
WholeStageCodegen (29)
HashAggregate [count] [count(1),h11_to_11_30,count]
InputAdapter
Exchange #12
WholeStageCodegen (19)
Exchange #18
WholeStageCodegen (28)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_store_sk,s_store_sk]
@ -127,8 +192,8 @@ BroadcastNestedLoopJoin
InputAdapter
ReusedExchange [hd_demo_sk] #2
InputAdapter
BroadcastExchange #13
WholeStageCodegen (17)
BroadcastExchange #19
WholeStageCodegen (26)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
@ -136,12 +201,13 @@ BroadcastNestedLoopJoin
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #4
BroadcastExchange #14
WholeStageCodegen (25)
HashAggregate [count] [count(1),h10_30_to_11,count]
InputAdapter
BroadcastExchange #20
WholeStageCodegen (34)
HashAggregate [count] [count(1),h11_30_to_12,count]
InputAdapter
Exchange #15
WholeStageCodegen (24)
Exchange #21
WholeStageCodegen (33)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_store_sk,s_store_sk]
@ -157,8 +223,8 @@ BroadcastNestedLoopJoin
InputAdapter
ReusedExchange [hd_demo_sk] #2
InputAdapter
BroadcastExchange #16
WholeStageCodegen (22)
BroadcastExchange #22
WholeStageCodegen (31)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
@ -166,12 +232,13 @@ BroadcastNestedLoopJoin
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #4
BroadcastExchange #17
WholeStageCodegen (30)
HashAggregate [count] [count(1),h11_to_11_30,count]
InputAdapter
BroadcastExchange #23
WholeStageCodegen (39)
HashAggregate [count] [count(1),h12_to_12_30,count]
InputAdapter
Exchange #18
WholeStageCodegen (29)
Exchange #24
WholeStageCodegen (38)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_store_sk,s_store_sk]
@ -187,8 +254,8 @@ BroadcastNestedLoopJoin
InputAdapter
ReusedExchange [hd_demo_sk] #2
InputAdapter
BroadcastExchange #19
WholeStageCodegen (27)
BroadcastExchange #25
WholeStageCodegen (36)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
@ -196,63 +263,3 @@ BroadcastNestedLoopJoin
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #4
BroadcastExchange #20
WholeStageCodegen (35)
HashAggregate [count] [count(1),h11_30_to_12,count]
InputAdapter
Exchange #21
WholeStageCodegen (34)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_store_sk,s_store_sk]
Project [ss_store_sk]
BroadcastHashJoin [ss_sold_time_sk,t_time_sk]
Project [ss_sold_time_sk,ss_store_sk]
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
Project [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
ReusedExchange [hd_demo_sk] #2
InputAdapter
BroadcastExchange #22
WholeStageCodegen (32)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #4
BroadcastExchange #23
WholeStageCodegen (40)
HashAggregate [count] [count(1),h12_to_12_30,count]
InputAdapter
Exchange #24
WholeStageCodegen (39)
HashAggregate [count,count]
Project
BroadcastHashJoin [ss_store_sk,s_store_sk]
Project [ss_store_sk]
BroadcastHashJoin [ss_sold_time_sk,t_time_sk]
Project [ss_sold_time_sk,ss_store_sk]
BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
Project [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
ColumnarToRow
InputAdapter
Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
InputAdapter
ReusedExchange [hd_demo_sk] #2
InputAdapter
BroadcastExchange #25
WholeStageCodegen (37)
Project [t_time_sk]
Filter [t_hour,t_minute,t_time_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour,t_minute]
InputAdapter
ReusedExchange [s_store_sk] #4

View file

@ -1,7 +1,7 @@
== Physical Plan ==
* Sort (52)
+- * Project (51)
+- BroadcastNestedLoopJoin Inner BuildRight (50)
+- * BroadcastNestedLoopJoin Inner BuildRight (50)
:- * HashAggregate (28)
: +- Exchange (27)
: +- * HashAggregate (26)
@ -175,7 +175,7 @@ Results [1]: [count#15]
Input [1]: [count#15]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#16]
(28) HashAggregate [codegen id : 5]
(28) HashAggregate [codegen id : 10]
Input [1]: [count#15]
Keys: []
Functions [1]: [count(1)]
@ -189,38 +189,38 @@ Location [not included in comparison]/{warehouse_dir}/web_sales]
PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)]
ReadSchema: struct<ws_sold_time_sk:int,ws_ship_hdemo_sk:int,ws_web_page_sk:int>
(30) ColumnarToRow [codegen id : 9]
(30) ColumnarToRow [codegen id : 8]
Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4]
(31) Filter [codegen id : 9]
(31) Filter [codegen id : 8]
Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4]
Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3))
(32) Project [codegen id : 9]
(32) Project [codegen id : 8]
Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3]
Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4]
(33) ReusedExchange [Reuses operator id: 9]
Output [1]: [wp_web_page_sk#5]
(34) BroadcastHashJoin [codegen id : 9]
(34) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ws_web_page_sk#3]
Right keys [1]: [wp_web_page_sk#5]
Join condition: None
(35) Project [codegen id : 9]
(35) Project [codegen id : 8]
Output [2]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2]
Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, wp_web_page_sk#5]
(36) ReusedExchange [Reuses operator id: 16]
Output [1]: [hd_demo_sk#8]
(37) BroadcastHashJoin [codegen id : 9]
(37) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ws_ship_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#8]
Join condition: None
(38) Project [codegen id : 9]
(38) Project [codegen id : 8]
Output [1]: [ws_sold_time_sk#1]
Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, hd_demo_sk#8]
@ -231,14 +231,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int>
(40) ColumnarToRow [codegen id : 8]
(40) ColumnarToRow [codegen id : 7]
Input [2]: [t_time_sk#11, t_hour#12]
(41) Filter [codegen id : 8]
(41) Filter [codegen id : 7]
Input [2]: [t_time_sk#11, t_hour#12]
Condition : (((isnotnull(t_hour#12) AND (t_hour#12 >= 19)) AND (t_hour#12 <= 20)) AND isnotnull(t_time_sk#11))
(42) Project [codegen id : 8]
(42) Project [codegen id : 7]
Output [1]: [t_time_sk#11]
Input [2]: [t_time_sk#11, t_hour#12]
@ -246,16 +246,16 @@ Input [2]: [t_time_sk#11, t_hour#12]
Input [1]: [t_time_sk#11]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19]
(44) BroadcastHashJoin [codegen id : 9]
(44) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ws_sold_time_sk#1]
Right keys [1]: [t_time_sk#11]
Join condition: None
(45) Project [codegen id : 9]
(45) Project [codegen id : 8]
Output: []
Input [2]: [ws_sold_time_sk#1, t_time_sk#11]
(46) HashAggregate [codegen id : 9]
(46) HashAggregate [codegen id : 8]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -266,7 +266,7 @@ Results [1]: [count#21]
Input [1]: [count#21]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22]
(48) HashAggregate [codegen id : 10]
(48) HashAggregate [codegen id : 9]
Input [1]: [count#21]
Keys: []
Functions [1]: [count(1)]
@ -277,14 +277,14 @@ Results [1]: [count(1)#23 AS pmc#24]
Input [1]: [pmc#24]
Arguments: IdentityBroadcastMode, [id=#25]
(50) BroadcastNestedLoopJoin
(50) BroadcastNestedLoopJoin [codegen id : 10]
Join condition: None
(51) Project [codegen id : 11]
(51) Project [codegen id : 10]
Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26]
Input [2]: [amc#18, pmc#24]
(52) Sort [codegen id : 11]
(52) Sort [codegen id : 10]
Input [1]: [am_pm_ratio#26]
Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0

View file

@ -1,55 +1,54 @@
WholeStageCodegen (11)
WholeStageCodegen (10)
Sort [am_pm_ratio]
Project [amc,pmc]
InputAdapter
BroadcastNestedLoopJoin
WholeStageCodegen (5)
HashAggregate [count] [count(1),amc,count]
InputAdapter
Exchange #1
WholeStageCodegen (4)
HashAggregate [count,count]
Project
BroadcastHashJoin [ws_sold_time_sk,t_time_sk]
Project [ws_sold_time_sk]
BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk]
Project [ws_sold_time_sk,ws_ship_hdemo_sk]
BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
Project [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk]
Filter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
InputAdapter
BroadcastExchange #2
WholeStageCodegen (1)
Project [wp_web_page_sk]
Filter [wp_char_count,wp_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_page [wp_web_page_sk,wp_char_count]
BroadcastNestedLoopJoin
HashAggregate [count] [count(1),amc,count]
InputAdapter
Exchange #1
WholeStageCodegen (4)
HashAggregate [count,count]
Project
BroadcastHashJoin [ws_sold_time_sk,t_time_sk]
Project [ws_sold_time_sk]
BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk]
Project [ws_sold_time_sk,ws_ship_hdemo_sk]
BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
Project [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk]
Filter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
InputAdapter
BroadcastExchange #3
WholeStageCodegen (2)
Project [hd_demo_sk]
Filter [hd_dep_count,hd_demo_sk]
BroadcastExchange #2
WholeStageCodegen (1)
Project [wp_web_page_sk]
Filter [wp_char_count,wp_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count]
Scan parquet default.web_page [wp_web_page_sk,wp_char_count]
InputAdapter
BroadcastExchange #4
WholeStageCodegen (3)
Project [t_time_sk]
Filter [t_hour,t_time_sk]
BroadcastExchange #3
WholeStageCodegen (2)
Project [hd_demo_sk]
Filter [hd_dep_count,hd_demo_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour]
Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count]
InputAdapter
BroadcastExchange #4
WholeStageCodegen (3)
Project [t_time_sk]
Filter [t_hour,t_time_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (10)
WholeStageCodegen (9)
HashAggregate [count] [count(1),pmc,count]
InputAdapter
Exchange #6
WholeStageCodegen (9)
WholeStageCodegen (8)
HashAggregate [count,count]
Project
BroadcastHashJoin [ws_sold_time_sk,t_time_sk]
@ -68,7 +67,7 @@ WholeStageCodegen (11)
ReusedExchange [hd_demo_sk] #3
InputAdapter
BroadcastExchange #7
WholeStageCodegen (8)
WholeStageCodegen (7)
Project [t_time_sk]
Filter [t_hour,t_time_sk]
ColumnarToRow

View file

@ -1,7 +1,7 @@
== Physical Plan ==
* Sort (52)
+- * Project (51)
+- BroadcastNestedLoopJoin Inner BuildRight (50)
+- * BroadcastNestedLoopJoin Inner BuildRight (50)
:- * HashAggregate (28)
: +- Exchange (27)
: +- * HashAggregate (26)
@ -175,7 +175,7 @@ Results [1]: [count#15]
Input [1]: [count#15]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#16]
(28) HashAggregate [codegen id : 5]
(28) HashAggregate [codegen id : 10]
Input [1]: [count#15]
Keys: []
Functions [1]: [count(1)]
@ -189,26 +189,26 @@ Location [not included in comparison]/{warehouse_dir}/web_sales]
PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)]
ReadSchema: struct<ws_sold_time_sk:int,ws_ship_hdemo_sk:int,ws_web_page_sk:int>
(30) ColumnarToRow [codegen id : 9]
(30) ColumnarToRow [codegen id : 8]
Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4]
(31) Filter [codegen id : 9]
(31) Filter [codegen id : 8]
Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4]
Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3))
(32) Project [codegen id : 9]
(32) Project [codegen id : 8]
Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3]
Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4]
(33) ReusedExchange [Reuses operator id: 9]
Output [1]: [hd_demo_sk#5]
(34) BroadcastHashJoin [codegen id : 9]
(34) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ws_ship_hdemo_sk#2]
Right keys [1]: [hd_demo_sk#5]
Join condition: None
(35) Project [codegen id : 9]
(35) Project [codegen id : 8]
Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3]
Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5]
@ -219,14 +219,14 @@ Location [not included in comparison]/{warehouse_dir}/time_dim]
PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)]
ReadSchema: struct<t_time_sk:int,t_hour:int>
(37) ColumnarToRow [codegen id : 7]
(37) ColumnarToRow [codegen id : 6]
Input [2]: [t_time_sk#8, t_hour#9]
(38) Filter [codegen id : 7]
(38) Filter [codegen id : 6]
Input [2]: [t_time_sk#8, t_hour#9]
Condition : (((isnotnull(t_hour#9) AND (t_hour#9 >= 19)) AND (t_hour#9 <= 20)) AND isnotnull(t_time_sk#8))
(39) Project [codegen id : 7]
(39) Project [codegen id : 6]
Output [1]: [t_time_sk#8]
Input [2]: [t_time_sk#8, t_hour#9]
@ -234,28 +234,28 @@ Input [2]: [t_time_sk#8, t_hour#9]
Input [1]: [t_time_sk#8]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19]
(41) BroadcastHashJoin [codegen id : 9]
(41) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ws_sold_time_sk#1]
Right keys [1]: [t_time_sk#8]
Join condition: None
(42) Project [codegen id : 9]
(42) Project [codegen id : 8]
Output [1]: [ws_web_page_sk#3]
Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#8]
(43) ReusedExchange [Reuses operator id: 23]
Output [1]: [wp_web_page_sk#11]
(44) BroadcastHashJoin [codegen id : 9]
(44) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [ws_web_page_sk#3]
Right keys [1]: [wp_web_page_sk#11]
Join condition: None
(45) Project [codegen id : 9]
(45) Project [codegen id : 8]
Output: []
Input [2]: [ws_web_page_sk#3, wp_web_page_sk#11]
(46) HashAggregate [codegen id : 9]
(46) HashAggregate [codegen id : 8]
Input: []
Keys: []
Functions [1]: [partial_count(1)]
@ -266,7 +266,7 @@ Results [1]: [count#21]
Input [1]: [count#21]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22]
(48) HashAggregate [codegen id : 10]
(48) HashAggregate [codegen id : 9]
Input [1]: [count#21]
Keys: []
Functions [1]: [count(1)]
@ -277,14 +277,14 @@ Results [1]: [count(1)#23 AS pmc#24]
Input [1]: [pmc#24]
Arguments: IdentityBroadcastMode, [id=#25]
(50) BroadcastNestedLoopJoin
(50) BroadcastNestedLoopJoin [codegen id : 10]
Join condition: None
(51) Project [codegen id : 11]
(51) Project [codegen id : 10]
Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26]
Input [2]: [amc#18, pmc#24]
(52) Sort [codegen id : 11]
(52) Sort [codegen id : 10]
Input [1]: [am_pm_ratio#26]
Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0

View file

@ -1,55 +1,54 @@
WholeStageCodegen (11)
WholeStageCodegen (10)
Sort [am_pm_ratio]
Project [amc,pmc]
InputAdapter
BroadcastNestedLoopJoin
WholeStageCodegen (5)
HashAggregate [count] [count(1),amc,count]
InputAdapter
Exchange #1
WholeStageCodegen (4)
HashAggregate [count,count]
Project
BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
Project [ws_web_page_sk]
BroadcastHashJoin [ws_sold_time_sk,t_time_sk]
Project [ws_sold_time_sk,ws_web_page_sk]
BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk]
Project [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk]
Filter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
InputAdapter
BroadcastExchange #2
WholeStageCodegen (1)
Project [hd_demo_sk]
Filter [hd_dep_count,hd_demo_sk]
ColumnarToRow
InputAdapter
Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count]
BroadcastNestedLoopJoin
HashAggregate [count] [count(1),amc,count]
InputAdapter
Exchange #1
WholeStageCodegen (4)
HashAggregate [count,count]
Project
BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
Project [ws_web_page_sk]
BroadcastHashJoin [ws_sold_time_sk,t_time_sk]
Project [ws_sold_time_sk,ws_web_page_sk]
BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk]
Project [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk]
Filter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
InputAdapter
BroadcastExchange #3
WholeStageCodegen (2)
Project [t_time_sk]
Filter [t_hour,t_time_sk]
BroadcastExchange #2
WholeStageCodegen (1)
Project [hd_demo_sk]
Filter [hd_dep_count,hd_demo_sk]
ColumnarToRow
InputAdapter
Scan parquet default.time_dim [t_time_sk,t_hour]
Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count]
InputAdapter
BroadcastExchange #4
WholeStageCodegen (3)
Project [wp_web_page_sk]
Filter [wp_char_count,wp_web_page_sk]
BroadcastExchange #3
WholeStageCodegen (2)
Project [t_time_sk]
Filter [t_hour,t_time_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_page [wp_web_page_sk,wp_char_count]
Scan parquet default.time_dim [t_time_sk,t_hour]
InputAdapter
BroadcastExchange #4
WholeStageCodegen (3)
Project [wp_web_page_sk]
Filter [wp_char_count,wp_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_page [wp_web_page_sk,wp_char_count]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (10)
WholeStageCodegen (9)
HashAggregate [count] [count(1),pmc,count]
InputAdapter
Exchange #6
WholeStageCodegen (9)
WholeStageCodegen (8)
HashAggregate [count,count]
Project
BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
@ -66,7 +65,7 @@ WholeStageCodegen (11)
ReusedExchange [hd_demo_sk] #2
InputAdapter
BroadcastExchange #7
WholeStageCodegen (7)
WholeStageCodegen (6)
Project [t_time_sk]
Filter [t_hour,t_time_sk]
ColumnarToRow

View file

@ -4,7 +4,7 @@ TakeOrderedAndProject (28)
+- Exchange (26)
+- * HashAggregate (25)
+- * Expand (24)
+- BroadcastNestedLoopJoin Inner BuildRight (23)
+- * BroadcastNestedLoopJoin Inner BuildRight (23)
:- * Project (19)
: +- * SortMergeJoin Inner (18)
: :- * Sort (12)
@ -105,12 +105,12 @@ Arguments: hashpartitioning(i_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#14]
Input [5]: [i_item_sk#9, i_brand#10, i_class#11, i_category#12, i_product_name#13]
Arguments: [i_item_sk#9 ASC NULLS FIRST], false, 0
(18) SortMergeJoin [codegen id : 6]
(18) SortMergeJoin [codegen id : 7]
Left keys [1]: [inv_item_sk#1]
Right keys [1]: [i_item_sk#9]
Join condition: None
(19) Project [codegen id : 6]
(19) Project [codegen id : 7]
Output [5]: [inv_quantity_on_hand#2, i_brand#10, i_class#11, i_category#12, i_product_name#13]
Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#9, i_brand#10, i_class#11, i_category#12, i_product_name#13]
@ -120,21 +120,21 @@ Batched: true
Location [not included in comparison]/{warehouse_dir}/warehouse]
ReadSchema: struct<>
(21) ColumnarToRow [codegen id : 7]
(21) ColumnarToRow [codegen id : 6]
Input: []
(22) BroadcastExchange
Input: []
Arguments: IdentityBroadcastMode, [id=#15]
(23) BroadcastNestedLoopJoin
(23) BroadcastNestedLoopJoin [codegen id : 7]
Join condition: None
(24) Expand [codegen id : 8]
(24) Expand [codegen id : 7]
Input [5]: [inv_quantity_on_hand#2, i_brand#10, i_class#11, i_category#12, i_product_name#13]
Arguments: [List(inv_quantity_on_hand#2, i_product_name#13, i_brand#10, i_class#11, i_category#12, 0), List(inv_quantity_on_hand#2, i_product_name#13, i_brand#10, i_class#11, null, 1), List(inv_quantity_on_hand#2, i_product_name#13, i_brand#10, null, null, 3), List(inv_quantity_on_hand#2, i_product_name#13, null, null, null, 7), List(inv_quantity_on_hand#2, null, null, null, null, 15)], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20]
(25) HashAggregate [codegen id : 8]
(25) HashAggregate [codegen id : 7]
Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20]
Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20]
Functions [1]: [partial_avg(inv_quantity_on_hand#2)]
@ -145,7 +145,7 @@ Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_gr
Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24]
Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, [id=#25]
(27) HashAggregate [codegen id : 9]
(27) HashAggregate [codegen id : 8]
Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24]
Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20]
Functions [1]: [avg(inv_quantity_on_hand#2)]

View file

@ -1,50 +1,49 @@
TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
WholeStageCodegen (9)
WholeStageCodegen (8)
HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count]
InputAdapter
Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1
WholeStageCodegen (8)
WholeStageCodegen (7)
HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count]
Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category]
InputAdapter
BroadcastNestedLoopJoin
WholeStageCodegen (6)
Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
SortMergeJoin [inv_item_sk,i_item_sk]
InputAdapter
WholeStageCodegen (3)
Sort [inv_item_sk]
InputAdapter
Exchange [inv_item_sk] #2
WholeStageCodegen (2)
Project [inv_item_sk,inv_quantity_on_hand]
BroadcastHashJoin [inv_date_sk,d_date_sk]
Filter [inv_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
ReusedExchange [d_date_sk] #3
InputAdapter
BroadcastExchange #3
WholeStageCodegen (1)
Project [d_date_sk]
Filter [d_month_seq,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_month_seq]
InputAdapter
WholeStageCodegen (5)
Sort [i_item_sk]
InputAdapter
Exchange [i_item_sk] #4
WholeStageCodegen (4)
Filter [i_item_sk]
BroadcastNestedLoopJoin
Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
SortMergeJoin [inv_item_sk,i_item_sk]
InputAdapter
WholeStageCodegen (3)
Sort [inv_item_sk]
InputAdapter
Exchange [inv_item_sk] #2
WholeStageCodegen (2)
Project [inv_item_sk,inv_quantity_on_hand]
BroadcastHashJoin [inv_date_sk,d_date_sk]
Filter [inv_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
Scan parquet default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
ReusedExchange [d_date_sk] #3
InputAdapter
BroadcastExchange #3
WholeStageCodegen (1)
Project [d_date_sk]
Filter [d_month_seq,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_month_seq]
InputAdapter
WholeStageCodegen (5)
Sort [i_item_sk]
InputAdapter
Exchange [i_item_sk] #4
WholeStageCodegen (4)
Filter [i_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (7)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
Scan parquet default.warehouse

View file

@ -4,7 +4,7 @@ TakeOrderedAndProject (25)
+- Exchange (23)
+- * HashAggregate (22)
+- * Expand (21)
+- BroadcastNestedLoopJoin Inner BuildRight (20)
+- * BroadcastNestedLoopJoin Inner BuildRight (20)
:- * Project (16)
: +- * BroadcastHashJoin Inner BuildRight (15)
: :- * Project (10)
@ -34,10 +34,10 @@ PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_s
PushedFilters: [IsNotNull(inv_item_sk)]
ReadSchema: struct<inv_item_sk:int,inv_quantity_on_hand:int>
(2) ColumnarToRow [codegen id : 3]
(2) ColumnarToRow [codegen id : 4]
Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3]
(3) Filter [codegen id : 3]
(3) Filter [codegen id : 4]
Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3]
Condition : isnotnull(inv_item_sk#1)
@ -63,12 +63,12 @@ Input [2]: [d_date_sk#5, d_month_seq#6]
Input [1]: [d_date_sk#5]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7]
(9) BroadcastHashJoin [codegen id : 3]
(9) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [inv_date_sk#3]
Right keys [1]: [d_date_sk#5]
Join condition: None
(10) Project [codegen id : 3]
(10) Project [codegen id : 4]
Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2]
Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5]
@ -90,12 +90,12 @@ Condition : isnotnull(i_item_sk#8)
Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13]
(15) BroadcastHashJoin [codegen id : 3]
(15) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [inv_item_sk#1]
Right keys [1]: [i_item_sk#8]
Join condition: None
(16) Project [codegen id : 3]
(16) Project [codegen id : 4]
Output [5]: [inv_quantity_on_hand#2, i_brand#9, i_class#10, i_category#11, i_product_name#12]
Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12]
@ -105,21 +105,21 @@ Batched: true
Location [not included in comparison]/{warehouse_dir}/warehouse]
ReadSchema: struct<>
(18) ColumnarToRow [codegen id : 4]
(18) ColumnarToRow [codegen id : 3]
Input: []
(19) BroadcastExchange
Input: []
Arguments: IdentityBroadcastMode, [id=#14]
(20) BroadcastNestedLoopJoin
(20) BroadcastNestedLoopJoin [codegen id : 4]
Join condition: None
(21) Expand [codegen id : 5]
(21) Expand [codegen id : 4]
Input [5]: [inv_quantity_on_hand#2, i_brand#9, i_class#10, i_category#11, i_product_name#12]
Arguments: [List(inv_quantity_on_hand#2, i_product_name#12, i_brand#9, i_class#10, i_category#11, 0), List(inv_quantity_on_hand#2, i_product_name#12, i_brand#9, i_class#10, null, 1), List(inv_quantity_on_hand#2, i_product_name#12, i_brand#9, null, null, 3), List(inv_quantity_on_hand#2, i_product_name#12, null, null, null, 7), List(inv_quantity_on_hand#2, null, null, null, null, 15)], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19]
(22) HashAggregate [codegen id : 5]
(22) HashAggregate [codegen id : 4]
Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19]
Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19]
Functions [1]: [partial_avg(inv_quantity_on_hand#2)]
@ -130,7 +130,7 @@ Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_gr
Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23]
Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, [id=#24]
(24) HashAggregate [codegen id : 6]
(24) HashAggregate [codegen id : 5]
Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23]
Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19]
Functions [1]: [avg(inv_quantity_on_hand#2)]

View file

@ -1,41 +1,40 @@
TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
WholeStageCodegen (6)
WholeStageCodegen (5)
HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count]
InputAdapter
Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1
WholeStageCodegen (5)
WholeStageCodegen (4)
HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count]
Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category]
InputAdapter
BroadcastNestedLoopJoin
WholeStageCodegen (3)
Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
BroadcastHashJoin [inv_item_sk,i_item_sk]
Project [inv_item_sk,inv_quantity_on_hand]
BroadcastHashJoin [inv_date_sk,d_date_sk]
Filter [inv_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
ReusedExchange [d_date_sk] #2
BroadcastNestedLoopJoin
Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
BroadcastHashJoin [inv_item_sk,i_item_sk]
Project [inv_item_sk,inv_quantity_on_hand]
BroadcastHashJoin [inv_date_sk,d_date_sk]
Filter [inv_item_sk]
ColumnarToRow
InputAdapter
BroadcastExchange #2
WholeStageCodegen (1)
Project [d_date_sk]
Filter [d_month_seq,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_month_seq]
Scan parquet default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
ReusedExchange [d_date_sk] #2
InputAdapter
BroadcastExchange #3
WholeStageCodegen (2)
Filter [i_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
BroadcastExchange #2
WholeStageCodegen (1)
Project [d_date_sk]
Filter [d_month_seq,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_month_seq]
InputAdapter
BroadcastExchange #3
WholeStageCodegen (2)
Filter [i_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
InputAdapter
BroadcastExchange #4
WholeStageCodegen (4)
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
Scan parquet default.warehouse

View file

@ -43,7 +43,7 @@ TakeOrderedAndProject (102)
: : : +- ReusedExchange (23)
: : +- ReusedExchange (26)
: :- * Project (53)
: : +- BroadcastNestedLoopJoin Inner BuildRight (52)
: : +- * BroadcastNestedLoopJoin Inner BuildRight (52)
: : :- * HashAggregate (42)
: : : +- Exchange (41)
: : : +- * HashAggregate (40)
@ -130,7 +130,7 @@ Input [2]: [d_date_sk#6, d_date#7]
(6) Filter [codegen id : 1]
Input [2]: [d_date_sk#6, d_date#7]
Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 10442)) AND (d_date#7 <= 10472)) AND isnotnull(d_date_sk#6))
Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6))
(7) Project [codegen id : 1]
Output [1]: [d_date_sk#6]
@ -297,7 +297,7 @@ Results [3]: [cs_call_center_sk#40, sum#46, sum#47]
Input [3]: [cs_call_center_sk#40, sum#46, sum#47]
Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48]
(42) HashAggregate [codegen id : 11]
(42) HashAggregate [codegen id : 14]
Input [3]: [cs_call_center_sk#40, sum#46, sum#47]
Keys [1]: [cs_call_center_sk#40]
Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))]
@ -311,22 +311,22 @@ Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)]
ReadSchema: struct<cr_return_amount:decimal(7,2),cr_net_loss:decimal(7,2)>
(44) ColumnarToRow [codegen id : 13]
(44) ColumnarToRow [codegen id : 12]
Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55]
(45) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#6]
(46) BroadcastHashJoin [codegen id : 13]
(46) BroadcastHashJoin [codegen id : 12]
Left keys [1]: [cr_returned_date_sk#55]
Right keys [1]: [d_date_sk#6]
Join condition: None
(47) Project [codegen id : 13]
(47) Project [codegen id : 12]
Output [2]: [cr_return_amount#53, cr_net_loss#54]
Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6]
(48) HashAggregate [codegen id : 13]
(48) HashAggregate [codegen id : 12]
Input [2]: [cr_return_amount#53, cr_net_loss#54]
Keys: []
Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))]
@ -337,7 +337,7 @@ Results [2]: [sum#58, sum#59]
Input [2]: [sum#58, sum#59]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60]
(50) HashAggregate [codegen id : 14]
(50) HashAggregate [codegen id : 13]
Input [2]: [sum#58, sum#59]
Keys: []
Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))]
@ -348,10 +348,10 @@ Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS re
Input [2]: [returns#63, profit_loss#64]
Arguments: IdentityBroadcastMode, [id=#65]
(52) BroadcastNestedLoopJoin
(52) BroadcastNestedLoopJoin [codegen id : 14]
Join condition: None
(53) Project [codegen id : 15]
(53) Project [codegen id : 14]
Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68]
Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64]
@ -363,22 +363,22 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_so
PushedFilters: [IsNotNull(ws_web_page_sk)]
ReadSchema: struct<ws_web_page_sk:int,ws_ext_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)>
(55) ColumnarToRow [codegen id : 18]
(55) ColumnarToRow [codegen id : 17]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72]
(56) Filter [codegen id : 18]
(56) Filter [codegen id : 17]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72]
Condition : isnotnull(ws_web_page_sk#69)
(57) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#6]
(58) BroadcastHashJoin [codegen id : 18]
(58) BroadcastHashJoin [codegen id : 17]
Left keys [1]: [ws_sold_date_sk#72]
Right keys [1]: [d_date_sk#6]
Join condition: None
(59) Project [codegen id : 18]
(59) Project [codegen id : 17]
Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71]
Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6]
@ -389,10 +389,10 @@ Location [not included in comparison]/{warehouse_dir}/web_page]
PushedFilters: [IsNotNull(wp_web_page_sk)]
ReadSchema: struct<wp_web_page_sk:int>
(61) ColumnarToRow [codegen id : 17]
(61) ColumnarToRow [codegen id : 16]
Input [1]: [wp_web_page_sk#73]
(62) Filter [codegen id : 17]
(62) Filter [codegen id : 16]
Input [1]: [wp_web_page_sk#73]
Condition : isnotnull(wp_web_page_sk#73)
@ -400,16 +400,16 @@ Condition : isnotnull(wp_web_page_sk#73)
Input [1]: [wp_web_page_sk#73]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74]
(64) BroadcastHashJoin [codegen id : 18]
(64) BroadcastHashJoin [codegen id : 17]
Left keys [1]: [ws_web_page_sk#69]
Right keys [1]: [wp_web_page_sk#73]
Join condition: None
(65) Project [codegen id : 18]
(65) Project [codegen id : 17]
Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
(66) HashAggregate [codegen id : 18]
(66) HashAggregate [codegen id : 17]
Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
Keys [1]: [wp_web_page_sk#73]
Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))]
@ -420,7 +420,7 @@ Results [3]: [wp_web_page_sk#73, sum#77, sum#78]
Input [3]: [wp_web_page_sk#73, sum#77, sum#78]
Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79]
(68) HashAggregate [codegen id : 23]
(68) HashAggregate [codegen id : 22]
Input [3]: [wp_web_page_sk#73, sum#77, sum#78]
Keys [1]: [wp_web_page_sk#73]
Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))]
@ -435,38 +435,38 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(w
PushedFilters: [IsNotNull(wr_web_page_sk)]
ReadSchema: struct<wr_web_page_sk:bigint,wr_return_amt:decimal(7,2),wr_net_loss:decimal(7,2)>
(70) ColumnarToRow [codegen id : 21]
(70) ColumnarToRow [codegen id : 20]
Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87]
(71) Filter [codegen id : 21]
(71) Filter [codegen id : 20]
Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87]
Condition : isnotnull(wr_web_page_sk#84)
(72) ReusedExchange [Reuses operator id: 63]
Output [1]: [wp_web_page_sk#88]
(73) BroadcastHashJoin [codegen id : 21]
(73) BroadcastHashJoin [codegen id : 20]
Left keys [1]: [wr_web_page_sk#84]
Right keys [1]: [cast(wp_web_page_sk#88 as bigint)]
Join condition: None
(74) Project [codegen id : 21]
(74) Project [codegen id : 20]
Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88]
Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88]
(75) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#6]
(76) BroadcastHashJoin [codegen id : 21]
(76) BroadcastHashJoin [codegen id : 20]
Left keys [1]: [wr_returned_date_sk#87]
Right keys [1]: [cast(d_date_sk#6 as bigint)]
Join condition: None
(77) Project [codegen id : 21]
(77) Project [codegen id : 20]
Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88]
Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6]
(78) HashAggregate [codegen id : 21]
(78) HashAggregate [codegen id : 20]
Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88]
Keys [1]: [wp_web_page_sk#88]
Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))]
@ -477,7 +477,7 @@ Results [3]: [wp_web_page_sk#88, sum#91, sum#92]
Input [3]: [wp_web_page_sk#88, sum#91, sum#92]
Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93]
(80) HashAggregate [codegen id : 22]
(80) HashAggregate [codegen id : 21]
Input [3]: [wp_web_page_sk#88, sum#91, sum#92]
Keys [1]: [wp_web_page_sk#88]
Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))]
@ -488,18 +488,18 @@ Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85)
Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98]
(82) BroadcastHashJoin [codegen id : 23]
(82) BroadcastHashJoin [codegen id : 22]
Left keys [1]: [wp_web_page_sk#73]
Right keys [1]: [wp_web_page_sk#88]
Join condition: None
(83) Project [codegen id : 23]
(83) Project [codegen id : 22]
Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102]
Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97]
(84) Union
(85) HashAggregate [codegen id : 24]
(85) HashAggregate [codegen id : 23]
Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39]
Keys [2]: [channel#36, id#37]
Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)]
@ -510,7 +510,7 @@ Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum
Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114]
Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115]
(87) HashAggregate [codegen id : 25]
(87) HashAggregate [codegen id : 24]
Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114]
Keys [2]: [channel#36, id#37]
Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)]
@ -520,14 +520,14 @@ Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sal
(88) ReusedExchange [Reuses operator id: unknown]
Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127]
(89) HashAggregate [codegen id : 50]
(89) HashAggregate [codegen id : 48]
Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127]
Keys [2]: [channel#36, id#37]
Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)]
Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131]
Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134]
(90) HashAggregate [codegen id : 50]
(90) HashAggregate [codegen id : 48]
Input [4]: [channel#36, sales#132, returns#133, profit#134]
Keys [1]: [channel#36]
Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)]
@ -538,7 +538,7 @@ Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, i
Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146]
Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147]
(92) HashAggregate [codegen id : 51]
(92) HashAggregate [codegen id : 49]
Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146]
Keys [1]: [channel#36]
Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)]
@ -548,14 +548,14 @@ Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(r
(93) ReusedExchange [Reuses operator id: unknown]
Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160]
(94) HashAggregate [codegen id : 76]
(94) HashAggregate [codegen id : 73]
Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160]
Keys [2]: [channel#36, id#37]
Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)]
Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164]
Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134]
(95) HashAggregate [codegen id : 76]
(95) HashAggregate [codegen id : 73]
Input [3]: [sales#132, returns#133, profit#134]
Keys: []
Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)]
@ -566,7 +566,7 @@ Results [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176]
Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177]
(97) HashAggregate [codegen id : 77]
(97) HashAggregate [codegen id : 74]
Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176]
Keys: []
Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)]
@ -575,7 +575,7 @@ Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#1
(98) Union
(99) HashAggregate [codegen id : 78]
(99) HashAggregate [codegen id : 75]
Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Functions: []
@ -586,7 +586,7 @@ Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186]
(101) HashAggregate [codegen id : 79]
(101) HashAggregate [codegen id : 76]
Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Functions: []

View file

@ -1,17 +1,17 @@
TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (79)
WholeStageCodegen (76)
HashAggregate [channel,id,sales,returns,profit]
InputAdapter
Exchange [channel,id,sales,returns,profit] #1
WholeStageCodegen (78)
WholeStageCodegen (75)
HashAggregate [channel,id,sales,returns,profit]
InputAdapter
Union
WholeStageCodegen (25)
WholeStageCodegen (24)
HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
Exchange [channel,id] #2
WholeStageCodegen (24)
WholeStageCodegen (23)
HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
Union
@ -70,30 +70,29 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedExchange [s_store_sk] #5
InputAdapter
ReusedExchange [d_date_sk] #4
WholeStageCodegen (15)
WholeStageCodegen (14)
Project [cs_call_center_sk,sales,returns,profit,profit_loss]
InputAdapter
BroadcastNestedLoopJoin
WholeStageCodegen (11)
HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum]
InputAdapter
Exchange [cs_call_center_sk] #8
WholeStageCodegen (10)
HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum]
Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #4
BroadcastNestedLoopJoin
HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum]
InputAdapter
Exchange [cs_call_center_sk] #8
WholeStageCodegen (10)
HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum]
Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #4
InputAdapter
BroadcastExchange #9
WholeStageCodegen (14)
WholeStageCodegen (13)
HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum]
InputAdapter
Exchange #10
WholeStageCodegen (13)
WholeStageCodegen (12)
HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum]
Project [cr_return_amount,cr_net_loss]
BroadcastHashJoin [cr_returned_date_sk,d_date_sk]
@ -103,13 +102,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #4
WholeStageCodegen (23)
WholeStageCodegen (22)
Project [wp_web_page_sk,sales,returns,profit,profit_loss]
BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk]
HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum]
InputAdapter
Exchange [wp_web_page_sk] #11
WholeStageCodegen (18)
WholeStageCodegen (17)
HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum]
Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
@ -124,18 +123,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedExchange [d_date_sk] #4
InputAdapter
BroadcastExchange #12
WholeStageCodegen (17)
WholeStageCodegen (16)
Filter [wp_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_page [wp_web_page_sk]
InputAdapter
BroadcastExchange #13
WholeStageCodegen (22)
WholeStageCodegen (21)
HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum]
InputAdapter
Exchange [wp_web_page_sk] #14
WholeStageCodegen (21)
WholeStageCodegen (20)
HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum]
Project [wr_return_amt,wr_net_loss,wp_web_page_sk]
BroadcastHashJoin [wr_returned_date_sk,d_date_sk]
@ -150,20 +149,20 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedExchange [wp_web_page_sk] #12
InputAdapter
ReusedExchange [d_date_sk] #4
WholeStageCodegen (51)
WholeStageCodegen (49)
HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
Exchange [channel] #15
WholeStageCodegen (50)
WholeStageCodegen (48)
HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #16
WholeStageCodegen (77)
WholeStageCodegen (74)
HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
Exchange #17
WholeStageCodegen (76)
WholeStageCodegen (73)
HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter

View file

@ -43,7 +43,7 @@ TakeOrderedAndProject (102)
: : : +- ReusedExchange (23)
: : +- ReusedExchange (26)
: :- * Project (53)
: : +- BroadcastNestedLoopJoin Inner BuildLeft (52)
: : +- * BroadcastNestedLoopJoin Inner BuildLeft (52)
: : :- BroadcastExchange (43)
: : : +- * HashAggregate (42)
: : : +- Exchange (41)
@ -130,7 +130,7 @@ Input [2]: [d_date_sk#6, d_date#7]
(6) Filter [codegen id : 1]
Input [2]: [d_date_sk#6, d_date#7]
Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 10442)) AND (d_date#7 <= 10472)) AND isnotnull(d_date_sk#6))
Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6))
(7) Project [codegen id : 1]
Output [1]: [d_date_sk#6]
@ -341,17 +341,17 @@ Results [2]: [sum#59, sum#60]
Input [2]: [sum#59, sum#60]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61]
(51) HashAggregate [codegen id : 14]
(51) HashAggregate
Input [2]: [sum#59, sum#60]
Keys: []
Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))]
Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63]
Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65]
(52) BroadcastNestedLoopJoin
(52) BroadcastNestedLoopJoin [codegen id : 14]
Join condition: None
(53) Project [codegen id : 15]
(53) Project [codegen id : 14]
Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68]
Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65]
@ -363,22 +363,22 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_so
PushedFilters: [IsNotNull(ws_web_page_sk)]
ReadSchema: struct<ws_web_page_sk:int,ws_ext_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)>
(55) ColumnarToRow [codegen id : 18]
(55) ColumnarToRow [codegen id : 17]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72]
(56) Filter [codegen id : 18]
(56) Filter [codegen id : 17]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72]
Condition : isnotnull(ws_web_page_sk#69)
(57) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#6]
(58) BroadcastHashJoin [codegen id : 18]
(58) BroadcastHashJoin [codegen id : 17]
Left keys [1]: [ws_sold_date_sk#72]
Right keys [1]: [d_date_sk#6]
Join condition: None
(59) Project [codegen id : 18]
(59) Project [codegen id : 17]
Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71]
Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6]
@ -389,10 +389,10 @@ Location [not included in comparison]/{warehouse_dir}/web_page]
PushedFilters: [IsNotNull(wp_web_page_sk)]
ReadSchema: struct<wp_web_page_sk:int>
(61) ColumnarToRow [codegen id : 17]
(61) ColumnarToRow [codegen id : 16]
Input [1]: [wp_web_page_sk#73]
(62) Filter [codegen id : 17]
(62) Filter [codegen id : 16]
Input [1]: [wp_web_page_sk#73]
Condition : isnotnull(wp_web_page_sk#73)
@ -400,16 +400,16 @@ Condition : isnotnull(wp_web_page_sk#73)
Input [1]: [wp_web_page_sk#73]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74]
(64) BroadcastHashJoin [codegen id : 18]
(64) BroadcastHashJoin [codegen id : 17]
Left keys [1]: [ws_web_page_sk#69]
Right keys [1]: [wp_web_page_sk#73]
Join condition: None
(65) Project [codegen id : 18]
(65) Project [codegen id : 17]
Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
(66) HashAggregate [codegen id : 18]
(66) HashAggregate [codegen id : 17]
Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73]
Keys [1]: [wp_web_page_sk#73]
Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))]
@ -420,7 +420,7 @@ Results [3]: [wp_web_page_sk#73, sum#77, sum#78]
Input [3]: [wp_web_page_sk#73, sum#77, sum#78]
Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79]
(68) HashAggregate [codegen id : 23]
(68) HashAggregate [codegen id : 22]
Input [3]: [wp_web_page_sk#73, sum#77, sum#78]
Keys [1]: [wp_web_page_sk#73]
Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))]
@ -435,38 +435,38 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(w
PushedFilters: [IsNotNull(wr_web_page_sk)]
ReadSchema: struct<wr_web_page_sk:bigint,wr_return_amt:decimal(7,2),wr_net_loss:decimal(7,2)>
(70) ColumnarToRow [codegen id : 21]
(70) ColumnarToRow [codegen id : 20]
Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87]
(71) Filter [codegen id : 21]
(71) Filter [codegen id : 20]
Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87]
Condition : isnotnull(wr_web_page_sk#84)
(72) ReusedExchange [Reuses operator id: 8]
Output [1]: [d_date_sk#6]
(73) BroadcastHashJoin [codegen id : 21]
(73) BroadcastHashJoin [codegen id : 20]
Left keys [1]: [wr_returned_date_sk#87]
Right keys [1]: [cast(d_date_sk#6 as bigint)]
Join condition: None
(74) Project [codegen id : 21]
(74) Project [codegen id : 20]
Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86]
Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6]
(75) ReusedExchange [Reuses operator id: 63]
Output [1]: [wp_web_page_sk#88]
(76) BroadcastHashJoin [codegen id : 21]
(76) BroadcastHashJoin [codegen id : 20]
Left keys [1]: [wr_web_page_sk#84]
Right keys [1]: [cast(wp_web_page_sk#88 as bigint)]
Join condition: None
(77) Project [codegen id : 21]
(77) Project [codegen id : 20]
Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88]
Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88]
(78) HashAggregate [codegen id : 21]
(78) HashAggregate [codegen id : 20]
Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88]
Keys [1]: [wp_web_page_sk#88]
Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))]
@ -477,7 +477,7 @@ Results [3]: [wp_web_page_sk#88, sum#91, sum#92]
Input [3]: [wp_web_page_sk#88, sum#91, sum#92]
Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93]
(80) HashAggregate [codegen id : 22]
(80) HashAggregate [codegen id : 21]
Input [3]: [wp_web_page_sk#88, sum#91, sum#92]
Keys [1]: [wp_web_page_sk#88]
Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))]
@ -488,18 +488,18 @@ Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85)
Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98]
(82) BroadcastHashJoin [codegen id : 23]
(82) BroadcastHashJoin [codegen id : 22]
Left keys [1]: [wp_web_page_sk#73]
Right keys [1]: [wp_web_page_sk#88]
Join condition: None
(83) Project [codegen id : 23]
(83) Project [codegen id : 22]
Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102]
Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97]
(84) Union
(85) HashAggregate [codegen id : 24]
(85) HashAggregate [codegen id : 23]
Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39]
Keys [2]: [channel#36, id#37]
Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)]
@ -510,7 +510,7 @@ Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum
Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114]
Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115]
(87) HashAggregate [codegen id : 25]
(87) HashAggregate [codegen id : 24]
Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114]
Keys [2]: [channel#36, id#37]
Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)]
@ -520,14 +520,14 @@ Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sal
(88) ReusedExchange [Reuses operator id: unknown]
Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127]
(89) HashAggregate [codegen id : 50]
(89) HashAggregate [codegen id : 48]
Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127]
Keys [2]: [channel#36, id#37]
Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)]
Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131]
Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134]
(90) HashAggregate [codegen id : 50]
(90) HashAggregate [codegen id : 48]
Input [4]: [channel#36, sales#132, returns#133, profit#134]
Keys [1]: [channel#36]
Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)]
@ -538,7 +538,7 @@ Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, i
Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146]
Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147]
(92) HashAggregate [codegen id : 51]
(92) HashAggregate [codegen id : 49]
Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146]
Keys [1]: [channel#36]
Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)]
@ -548,14 +548,14 @@ Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(r
(93) ReusedExchange [Reuses operator id: unknown]
Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160]
(94) HashAggregate [codegen id : 76]
(94) HashAggregate [codegen id : 73]
Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160]
Keys [2]: [channel#36, id#37]
Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)]
Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164]
Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134]
(95) HashAggregate [codegen id : 76]
(95) HashAggregate [codegen id : 73]
Input [3]: [sales#132, returns#133, profit#134]
Keys: []
Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)]
@ -566,7 +566,7 @@ Results [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176]
Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177]
(97) HashAggregate [codegen id : 77]
(97) HashAggregate [codegen id : 74]
Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176]
Keys: []
Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)]
@ -575,7 +575,7 @@ Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#1
(98) Union
(99) HashAggregate [codegen id : 78]
(99) HashAggregate [codegen id : 75]
Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Functions: []
@ -586,7 +586,7 @@ Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186]
(101) HashAggregate [codegen id : 79]
(101) HashAggregate [codegen id : 76]
Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121]
Functions: []

View file

@ -1,17 +1,17 @@
TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (79)
WholeStageCodegen (76)
HashAggregate [channel,id,sales,returns,profit]
InputAdapter
Exchange [channel,id,sales,returns,profit] #1
WholeStageCodegen (78)
WholeStageCodegen (75)
HashAggregate [channel,id,sales,returns,profit]
InputAdapter
Union
WholeStageCodegen (25)
WholeStageCodegen (24)
HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
Exchange [channel,id] #2
WholeStageCodegen (24)
WholeStageCodegen (23)
HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
Union
@ -70,10 +70,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedExchange [d_date_sk] #4
InputAdapter
ReusedExchange [s_store_sk] #5
WholeStageCodegen (15)
WholeStageCodegen (14)
Project [cs_call_center_sk,sales,returns,profit,profit_loss]
InputAdapter
BroadcastNestedLoopJoin
BroadcastNestedLoopJoin
InputAdapter
BroadcastExchange #8
WholeStageCodegen (11)
HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum]
@ -89,27 +89,26 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #4
WholeStageCodegen (14)
HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum]
InputAdapter
Exchange #10
WholeStageCodegen (13)
HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum]
Project [cr_return_amount,cr_net_loss]
BroadcastHashJoin [cr_returned_date_sk,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #4
WholeStageCodegen (23)
HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum]
InputAdapter
Exchange #10
WholeStageCodegen (13)
HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum]
Project [cr_return_amount,cr_net_loss]
BroadcastHashJoin [cr_returned_date_sk,d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
ReusedExchange [d_date_sk] #4
WholeStageCodegen (22)
Project [wp_web_page_sk,sales,returns,profit,profit_loss]
BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk]
HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum]
InputAdapter
Exchange [wp_web_page_sk] #11
WholeStageCodegen (18)
WholeStageCodegen (17)
HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum]
Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
@ -124,18 +123,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedExchange [d_date_sk] #4
InputAdapter
BroadcastExchange #12
WholeStageCodegen (17)
WholeStageCodegen (16)
Filter [wp_web_page_sk]
ColumnarToRow
InputAdapter
Scan parquet default.web_page [wp_web_page_sk]
InputAdapter
BroadcastExchange #13
WholeStageCodegen (22)
WholeStageCodegen (21)
HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum]
InputAdapter
Exchange [wp_web_page_sk] #14
WholeStageCodegen (21)
WholeStageCodegen (20)
HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum]
Project [wr_return_amt,wr_net_loss,wp_web_page_sk]
BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk]
@ -150,20 +149,20 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ReusedExchange [d_date_sk] #4
InputAdapter
ReusedExchange [wp_web_page_sk] #12
WholeStageCodegen (51)
WholeStageCodegen (49)
HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
Exchange [channel] #15
WholeStageCodegen (50)
WholeStageCodegen (48)
HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #16
WholeStageCodegen (77)
WholeStageCodegen (74)
HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter
Exchange #17
WholeStageCodegen (76)
WholeStageCodegen (73)
HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty]
InputAdapter

View file

@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAnd
import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite
import org.apache.spark.sql.execution.aggregate.HashAggregateExec
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, ShuffledHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession
@ -92,6 +92,46 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession
Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4)))
}
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"))
val df3 = spark.range(2).select($"id".as("k3"))
Seq(true, false).foreach { codegenEnabled =>
withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegenEnabled.toString) {
// test broadcast nested loop join without condition
val oneJoinDF = df1.join(df2)
var hasJoinInCodegen = oneJoinDF.queryExecution.executedPlan.collect {
case WholeStageCodegenExec(_ : BroadcastNestedLoopJoinExec) => true
}.size === 1
assert(hasJoinInCodegen == codegenEnabled)
checkAnswer(oneJoinDF,
Seq(Row(0, 0), Row(0, 1), Row(0, 2), Row(1, 0), Row(1, 1), Row(1, 2),
Row(2, 0), Row(2, 1), Row(2, 2), Row(3, 0), Row(3, 1), Row(3, 2)))
// test broadcast nested loop join with condition
val oneJoinDFWithCondition = df1.join(df2, $"k1" + 1 =!= $"k2")
hasJoinInCodegen = oneJoinDFWithCondition.queryExecution.executedPlan.collect {
case WholeStageCodegenExec(_ : BroadcastNestedLoopJoinExec) => true
}.size === 1
assert(hasJoinInCodegen == codegenEnabled)
checkAnswer(oneJoinDFWithCondition,
Seq(Row(0, 0), Row(0, 2), Row(1, 0), Row(1, 1), Row(2, 0), Row(2, 1),
Row(2, 2), Row(3, 0), Row(3, 1), Row(3, 2)))
// test two broadcast nested loop joins
val twoJoinsDF = df1.join(df2, $"k1" < $"k2").crossJoin(df3)
hasJoinInCodegen = twoJoinsDF.queryExecution.executedPlan.collect {
case WholeStageCodegenExec(BroadcastNestedLoopJoinExec(
_: BroadcastNestedLoopJoinExec, _, _, _, _)) => true
}.size === 1
assert(hasJoinInCodegen == codegenEnabled)
checkAnswer(twoJoinsDF,
Seq(Row(0, 1, 0), Row(0, 2, 0), Row(1, 2, 0), Row(0, 1, 1), Row(0, 2, 1), Row(1, 2, 1)))
}
}
}
test("Sort should be included in WholeStageCodegen") {
val df = spark.range(3, 0, -1).toDF().sort(col("id"))
val plan = df.queryExecution.executedPlan

View file

@ -166,6 +166,19 @@ object JoinBenchmark extends SqlBasedBenchmark {
}
}
def broadcastNestedLoopJoin(): Unit = {
val N = 20 << 20
val M = 1 << 4
val dim = broadcast(spark.range(M).selectExpr("id as k", "cast(id as string) as v"))
codegenBenchmark("broadcast nested loop join", N) {
val df = spark.range(N).join(dim)
assert(df.queryExecution.sparkPlan.find(
_.isInstanceOf[BroadcastNestedLoopJoinExec]).isDefined)
df.noop()
}
}
override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
runBenchmark("Join Benchmark") {
broadcastHashJoinLongKey()
@ -178,6 +191,7 @@ object JoinBenchmark extends SqlBasedBenchmark {
sortMergeJoin()
sortMergeJoinWithDuplicates()
shuffleHashJoin()
broadcastNestedLoopJoin()
}
}
}