From 39542bb81f8570219770bb6533c077f44f6cbd2a Mon Sep 17 00:00:00 2001 From: hezuojiao Date: Mon, 22 Mar 2021 13:06:12 -0700 Subject: [PATCH 01/24] [SPARK-34790][CORE] Disable fetching shuffle blocks in batch when io encryption is enabled ### What changes were proposed in this pull request? This patch proposes to disable fetching shuffle blocks in batch when io encryption is enabled. Adaptive Query Execution fetch contiguous shuffle blocks for the same map task in batch to reduce IO and improve performance. However, we found that batch fetching is incompatible with io encryption. ### Why are the changes needed? Before this patch, we set `spark.io.encryption.enabled` to true, then run some queries which coalesced partitions by AEQ, may got following error message: ```14:05:52.638 WARN org.apache.spark.scheduler.TaskSetManager: Lost task 1.0 in stage 2.0 (TID 3) (11.240.37.88 executor driver): FetchFailed(BlockManagerId(driver, 11.240.37.88, 63574, None), shuffleId=0, mapIndex=0, mapId=0, reduceId=2, message= org.apache.spark.shuffle.FetchFailedException: Stream is corrupted at org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:772) at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:845) at java.io.BufferedInputStream.fill(BufferedInputStream.java:246) at java.io.BufferedInputStream.read(BufferedInputStream.java:265) at java.io.DataInputStream.readInt(DataInputStream.java:387) at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.readSize(UnsafeRowSerializer.scala:113) at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:129) at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:110) at scala.collection.Iterator$$anon$11.next(Iterator.scala:494) at scala.collection.Iterator$$anon$10.next(Iterator.scala:459) at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:29) at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:40) at scala.collection.Iterator$$anon$10.next(Iterator.scala:459) at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:345) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373) at org.apache.spark.rdd.RDD.iterator(RDD.scala:337) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:131) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:498) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: java.io.IOException: Stream is corrupted at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:200) at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:226) at net.jpountz.lz4.LZ4BlockInputStream.read(LZ4BlockInputStream.java:157) at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:841) ... 25 more ) ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? New tests. Closes #31898 from hezuojiao/fetch_shuffle_in_batch. Authored-by: hezuojiao Signed-off-by: Dongjoon Hyun --- .../shuffle/BlockStoreShuffleReader.scala | 6 +++-- .../apache/spark/sql/internal/SQLConf.scala | 4 +-- .../CoalesceShufflePartitionsSuite.scala | 25 ++++++++++++++++++- 3 files changed, 30 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index f1afc6c8954bb..e44d96e273504 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -51,15 +51,17 @@ private[spark] class BlockStoreShuffleReader[K, C]( true } val useOldFetchProtocol = conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) + // SPARK-34790: Fetching continuous blocks in batch is incompatible with io encryption. + val ioEncryption = conf.get(config.IO_ENCRYPTION_ENABLED) val doBatchFetch = shouldBatchFetch && serializerRelocatable && - (!compressed || codecConcatenation) && !useOldFetchProtocol + (!compressed || codecConcatenation) && !useOldFetchProtocol && !ioEncryption if (shouldBatchFetch && !doBatchFetch) { logDebug("The feature tag of continuous shuffle block fetching is set to true, but " + "we can not enable the feature because other conditions are not satisfied. " + s"Shuffle compress: $compressed, serializer relocatable: $serializerRelocatable, " + s"codec concatenation: $codecConcatenation, use old shuffle fetch protocol: " + - s"$useOldFetchProtocol.") + s"$useOldFetchProtocol, io encryption: $ioEncryption.") } doBatchFetch } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 610f436050b04..130af3ae964c7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -500,8 +500,8 @@ object SQLConf { "reduce IO and improve performance. Note, multiple contiguous blocks exist in single " + s"fetch request only happen when '${ADAPTIVE_EXECUTION_ENABLED.key}' and " + s"'${COALESCE_PARTITIONS_ENABLED.key}' are both true. This feature also depends " + - "on a relocatable serializer, the concatenation support codec in use and the new version " + - "shuffle fetch protocol.") + "on a relocatable serializer, the concatenation support codec in use, the new version " + + "shuffle fetch protocol and io encryption is disabled.") .version("3.0.0") .booleanConf .createWithDefault(true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala index e562981022306..6b9f49409e485 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ import org.apache.spark.sql.execution.adaptive._ @@ -57,15 +58,18 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl def withSparkSession( f: SparkSession => Unit, targetPostShuffleInputSize: Int, - minNumPostShufflePartitions: Option[Int]): Unit = { + minNumPostShufflePartitions: Option[Int], + enableIOEncryption: Boolean = false): Unit = { val sparkConf = new SparkConf(false) .setMaster("local[*]") .setAppName("test") .set(UI_ENABLED, false) + .set(IO_ENCRYPTION_ENABLED, enableIOEncryption) .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") .set(SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + .set(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") .set( SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key, @@ -408,6 +412,25 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl } withSparkSession(test, 100, None) } + + test("SPARK-34790: enable IO encryption in AQE partition coalescing") { + val test: SparkSession => Unit = { spark: SparkSession => + val ds = spark.range(0, 100, 1, numInputPartitions) + val resultDf = ds.repartition(ds.col("id")) + resultDf.collect() + + val finalPlan = resultDf.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + assert( + finalPlan.collect { + case r @ CoalescedShuffleReader() => r + }.isDefinedAt(0)) + } + Seq(true, false).foreach { enableIOEncryption => + // Before SPARK-34790, it will throw an exception when io encryption enabled. + withSparkSession(test, Int.MaxValue, None, enableIOEncryption) + } + } } object CoalescedShuffleReader { From e768eaa9085de48b6594386ee0f60556ab3c44c3 Mon Sep 17 00:00:00 2001 From: linzebing Date: Tue, 23 Mar 2021 07:11:57 +0000 Subject: [PATCH 02/24] [SPARK-34707][SQL] Code-gen broadcast nested loop join (left outer/right outer) ### What changes were proposed in this pull request? This PR is to add code-gen support for left outer (build right) and right outer (build left). Reference: `BroadcastNestedLoopJoinExec.codegenInner()` and `BroadcastNestedLoopJoinExec.outerJoin()` ### Why are the changes needed? Improve query CPU performance. Tested with a simple query: ```scala val N = 20 << 20 val M = 1 << 4 val dim = broadcast(spark.range(M).selectExpr("id as k2")) codegenBenchmark("left outer broadcast nested loop join", N) { val df = spark.range(N).selectExpr(s"id as k1").join( dim, col("k1") + 1 <= col("k2"), "left_outer") assert(df.queryExecution.sparkPlan.find( _.isInstanceOf[BroadcastNestedLoopJoinExec]).isDefined) df.noop() } ``` Seeing 2x run time improvement: ``` Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7 Intel(R) Core(TM) i9-9980HK CPU 2.40GHz left outer broadcast nested loop join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ left outer broadcast nested loop join wholestage off 3024 3698 953 6.9 144.2 1.0X left outer broadcast nested loop join wholestage on 1512 1659 172 13.9 72.1 2.0X ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Changed existing unit tests in `OuterJoinSuite` to cover codegen use cases. Added unit test in WholeStageCodegenSuite.scala to make sure code-gen for broadcast nested loop join is taking effect, and test for multiple join case as well. Example query: ```scala val df1 = spark.range(4).select($"id".as("k1")) val df2 = spark.range(3).select($"id".as("k2")) df1.join(df2, $"k1" + 1 <= $"k2", "left_outer").explain("codegen") ``` Example generated code (`bnlj_doConsume_0` method): ```java == Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:210(0.32% used); numInnerClasses:0) == *(2) BroadcastNestedLoopJoin BuildRight, LeftOuter, ((k1#2L + 1) <= k2#6L) :- *(2) Project [id#0L AS k1#2L] : +- *(2) Range (0, 4, step=1, splits=16) +- BroadcastExchange IdentityBroadcastMode, [id=#22] +- *(1) Project [id#4L AS k2#6L] +- *(1) Range (0, 3, step=1, splits=16) 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 */ boolean bnlj_foundMatch_0 = false; /* 038 */ for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) { /* 039 */ UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0]; /* 040 */ boolean bnlj_shouldOutputRow_0 = false; /* 041 */ /* 042 */ boolean bnlj_isNull_2 = true; /* 043 */ long bnlj_value_2 = -1L; /* 044 */ if (bnlj_buildRow_0 != null) { /* 045 */ long bnlj_value_1 = bnlj_buildRow_0.getLong(0); /* 046 */ bnlj_isNull_2 = false; /* 047 */ bnlj_value_2 = bnlj_value_1; /* 048 */ } /* 049 */ /* 050 */ long bnlj_value_4 = -1L; /* 051 */ /* 052 */ bnlj_value_4 = bnlj_expr_0_0 + 1L; /* 053 */ /* 054 */ boolean bnlj_value_3 = false; /* 055 */ bnlj_value_3 = bnlj_value_4 <= bnlj_value_2; /* 056 */ if (!(false || !bnlj_value_3)) /* 057 */ { /* 058 */ bnlj_shouldOutputRow_0 = true; /* 059 */ bnlj_foundMatch_0 = true; /* 060 */ } /* 061 */ if (bnlj_arrayIndex_0 == bnlj_buildRowArray_0.length - 1 && !bnlj_foundMatch_0) { /* 062 */ bnlj_buildRow_0 = null; /* 063 */ bnlj_shouldOutputRow_0 = true; /* 064 */ } /* 065 */ if (bnlj_shouldOutputRow_0) { /* 066 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1); /* 067 */ /* 068 */ boolean bnlj_isNull_9 = true; /* 069 */ long bnlj_value_9 = -1L; /* 070 */ if (bnlj_buildRow_0 != null) { /* 071 */ long bnlj_value_8 = bnlj_buildRow_0.getLong(0); /* 072 */ bnlj_isNull_9 = false; /* 073 */ bnlj_value_9 = bnlj_value_8; /* 074 */ } /* 075 */ range_mutableStateArray_0[3].reset(); /* 076 */ /* 077 */ range_mutableStateArray_0[3].zeroOutNullBytes(); /* 078 */ /* 079 */ range_mutableStateArray_0[3].write(0, bnlj_expr_0_0); /* 080 */ /* 081 */ if (bnlj_isNull_9) { /* 082 */ range_mutableStateArray_0[3].setNullAt(1); /* 083 */ } else { /* 084 */ range_mutableStateArray_0[3].write(1, bnlj_value_9); /* 085 */ } /* 086 */ append((range_mutableStateArray_0[3].getRow()).copy()); /* 087 */ /* 088 */ } /* 089 */ } /* 090 */ /* 091 */ } /* 092 */ /* 093 */ private void initRange(int idx) { /* 094 */ java.math.BigInteger index = java.math.BigInteger.valueOf(idx); /* 095 */ java.math.BigInteger numSlice = java.math.BigInteger.valueOf(16L); /* 096 */ java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L); /* 097 */ java.math.BigInteger step = java.math.BigInteger.valueOf(1L); /* 098 */ java.math.BigInteger start = java.math.BigInteger.valueOf(0L); /* 099 */ long partitionEnd; /* 100 */ /* 101 */ java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start); /* 102 */ if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) { /* 103 */ range_nextIndex_0 = Long.MAX_VALUE; /* 104 */ } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) { /* 105 */ range_nextIndex_0 = Long.MIN_VALUE; /* 106 */ } else { /* 107 */ range_nextIndex_0 = st.longValue(); /* 108 */ } /* 109 */ range_batchEnd_0 = range_nextIndex_0; /* 110 */ /* 111 */ java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice) /* 112 */ .multiply(step).add(start); /* 113 */ if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) { /* 114 */ partitionEnd = Long.MAX_VALUE; /* 115 */ } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) { /* 116 */ partitionEnd = Long.MIN_VALUE; /* 117 */ } else { /* 118 */ partitionEnd = end.longValue(); /* 119 */ } /* 120 */ /* 121 */ java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract( /* 122 */ java.math.BigInteger.valueOf(range_nextIndex_0)); /* 123 */ range_numElementsTodo_0 = startToEnd.divide(step).longValue(); /* 124 */ if (range_numElementsTodo_0 < 0) { /* 125 */ range_numElementsTodo_0 = 0; /* 126 */ } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) { /* 127 */ range_numElementsTodo_0++; /* 128 */ } /* 129 */ } /* 130 */ /* 131 */ protected void processNext() throws java.io.IOException { /* 132 */ // initialize Range /* 133 */ if (!range_initRange_0) { /* 134 */ range_initRange_0 = true; /* 135 */ initRange(partitionIndex); /* 136 */ } /* 137 */ /* 138 */ while (true) { /* 139 */ if (range_nextIndex_0 == range_batchEnd_0) { /* 140 */ long range_nextBatchTodo_0; /* 141 */ if (range_numElementsTodo_0 > 1000L) { /* 142 */ range_nextBatchTodo_0 = 1000L; /* 143 */ range_numElementsTodo_0 -= 1000L; /* 144 */ } else { /* 145 */ range_nextBatchTodo_0 = range_numElementsTodo_0; /* 146 */ range_numElementsTodo_0 = 0; /* 147 */ if (range_nextBatchTodo_0 == 0) break; /* 148 */ } /* 149 */ range_batchEnd_0 += range_nextBatchTodo_0 * 1L; /* 150 */ } /* 151 */ /* 152 */ int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L); /* 153 */ for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) { /* 154 */ long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0; /* 155 */ /* 156 */ // common sub-expressions /* 157 */ /* 158 */ bnlj_doConsume_0(range_value_0); /* 159 */ /* 160 */ if (shouldStop()) { /* 161 */ range_nextIndex_0 = range_value_0 + 1L; /* 162 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1); /* 163 */ range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1); /* 164 */ return; /* 165 */ } /* 166 */ /* 167 */ } /* 168 */ range_nextIndex_0 = range_batchEnd_0; /* 169 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0); /* 170 */ range_inputMetrics_0.incRecordsRead(range_localEnd_0); /* 171 */ range_taskContext_0.killTaskIfInterrupted(); /* 172 */ } /* 173 */ } /* 174 */ /* 175 */ } ``` Closes #31931 from linzebing/code-left-right-outer. Authored-by: linzebing Signed-off-by: Wenchen Fan --- .../joins/BroadcastNestedLoopJoinExec.scala | 47 ++++++++++++++++++- .../execution/WholeStageCodegenSuite.scala | 47 +++++++++++++++++++ .../sql/execution/joins/OuterJoinSuite.scala | 4 +- .../execution/metric/SQLMetricsSuite.scala | 6 +-- 4 files changed, 98 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index 482c3a3091f86..fa1a57a8ae3a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -396,7 +396,8 @@ case class BroadcastNestedLoopJoinExec( } override def supportCodegen: Boolean = (joinType, buildSide) match { - case (_: InnerLike, _) | (LeftSemi | LeftAnti, BuildRight) => true + case (_: InnerLike, _) | (LeftOuter, BuildRight) | (RightOuter, BuildLeft) | + (LeftSemi | LeftAnti, BuildRight) => true case _ => false } @@ -413,6 +414,7 @@ case class BroadcastNestedLoopJoinExec( override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { (joinType, buildSide) match { case (_: InnerLike, _) => codegenInner(ctx, input) + case (LeftOuter, BuildRight) | (RightOuter, BuildLeft) => codegenOuter(ctx, input) case (LeftSemi, BuildRight) => codegenLeftExistence(ctx, input, exists = true) case (LeftAnti, BuildRight) => codegenLeftExistence(ctx, input, exists = false) case _ => @@ -458,6 +460,49 @@ case class BroadcastNestedLoopJoinExec( """.stripMargin } + private def codegenOuter(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (buildRowArray, buildRowArrayTerm) = prepareBroadcast(ctx) + val (buildRow, checkCondition, _) = getJoinCondition(ctx, input, streamed, broadcast) + val buildVars = genBuildSideVars(ctx, buildRow, broadcast) + + val resultVars = buildSide match { + case BuildLeft => buildVars ++ input + case BuildRight => input ++ buildVars + } + val arrayIndex = ctx.freshName("arrayIndex") + val shouldOutputRow = ctx.freshName("shouldOutputRow") + val foundMatch = ctx.freshName("foundMatch") + val numOutput = metricTerm(ctx, "numOutputRows") + + if (buildRowArray.isEmpty) { + s""" + |UnsafeRow $buildRow = null; + |$numOutput.add(1); + |${consume(ctx, resultVars)} + """.stripMargin + } else { + s""" + |boolean $foundMatch = false; + |for (int $arrayIndex = 0; $arrayIndex < $buildRowArrayTerm.length; $arrayIndex++) { + | UnsafeRow $buildRow = (UnsafeRow) $buildRowArrayTerm[$arrayIndex]; + | boolean $shouldOutputRow = false; + | $checkCondition { + | $shouldOutputRow = true; + | $foundMatch = true; + | } + | if ($arrayIndex == $buildRowArrayTerm.length - 1 && !$foundMatch) { + | $buildRow = null; + | $shouldOutputRow = true; + | } + | if ($shouldOutputRow) { + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + |} + """.stripMargin + } + } + private def codegenLeftExistence( ctx: CodegenContext, input: Seq[ExprCode], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 8246bca1893a9..b66308c4f880f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -211,6 +211,53 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession } } + test("Left/Right outer 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")) + val df4 = spark.range(0).select($"id".as("k4")) + + Seq(true, false).foreach { codegenEnabled => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegenEnabled.toString) { + // test left outer join + val leftOuterJoinDF = df1.join(df2, $"k1" > $"k2", "left_outer") + var hasJoinInCodegen = leftOuterJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: BroadcastNestedLoopJoinExec) => true + }.size === 1 + assert(hasJoinInCodegen == codegenEnabled) + checkAnswer(leftOuterJoinDF, + Seq(Row(0, null), Row(1, 0), Row(2, 0), Row(2, 1), Row(3, 0), Row(3, 1), Row(3, 2))) + + // test right outer join + val rightOuterJoinDF = df1.join(df2, $"k1" < $"k2", "right_outer") + hasJoinInCodegen = rightOuterJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: BroadcastNestedLoopJoinExec) => true + }.size === 1 + assert(hasJoinInCodegen == codegenEnabled) + checkAnswer(rightOuterJoinDF, Seq(Row(null, 0), Row(0, 1), Row(0, 2), Row(1, 2))) + + // test a combination of left outer and right outer joins + val twoJoinsDF = df1.join(df2, $"k1" > $"k2" + 1, "right_outer") + .join(df3, $"k1" <= $"k3", "left_outer") + hasJoinInCodegen = twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(BroadcastNestedLoopJoinExec( + _: BroadcastNestedLoopJoinExec, _, _, _, _)) => true + }.size === 1 + assert(hasJoinInCodegen == codegenEnabled) + checkAnswer(twoJoinsDF, + Seq(Row(2, 0, null), Row(3, 0, null), Row(3, 1, null), Row(null, 2, null))) + + // test build side is empty + val buildSideIsEmptyDF = df3.join(df4, $"k3" > $"k4", "left_outer") + hasJoinInCodegen = buildSideIsEmptyDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: BroadcastNestedLoopJoinExec) => true + }.size === 1 + assert(hasJoinInCodegen == codegenEnabled) + checkAnswer(buildSideIsEmptyDF, Seq(Row(0, null), Row(1, null))) + } + } + } + test("Left semi/anti BroadcastNestedLoopJoinExec should be included in WholeStageCodegen") { val df1 = spark.range(4).select($"id".as("k1")) val df2 = spark.range(3).select($"id".as("k2")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala index 150d40d0301fc..810eeea5b9a60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -149,7 +149,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { } } - test(s"$testName using BroadcastNestedLoopJoin build left") { + testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastNestedLoopJoin build left") { _ => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => BroadcastNestedLoopJoinExec(left, right, BuildLeft, joinType, Some(condition)), @@ -158,7 +158,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { } } - test(s"$testName using BroadcastNestedLoopJoin build right") { + testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastNestedLoopJoin build right") { _ => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => BroadcastNestedLoopJoinExec(left, right, BuildRight, joinType, Some(condition)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index dd99368e3a87b..50f980643d2d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -452,11 +452,11 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils "testData2.a * testDataForJoin.a != testData2.a + testDataForJoin.a" val rightQuery = "SELECT * FROM testData2 RIGHT JOIN testDataForJoin ON " + "testData2.a * testDataForJoin.a != testData2.a + testDataForJoin.a" - Seq((leftQuery, false), (rightQuery, false), (leftQuery, true), (rightQuery, true)) - .foreach { case (query, enableWholeStage) => + Seq((leftQuery, 0L, false), (rightQuery, 0L, false), (leftQuery, 1L, true), + (rightQuery, 1L, true)).foreach { case (query, nodeId, enableWholeStage) => val df = spark.sql(query) testSparkPlanMetrics(df, 2, Map( - 0L -> (("BroadcastNestedLoopJoin", Map( + nodeId -> (("BroadcastNestedLoopJoin", Map( "number of output rows" -> 12L)))), enableWholeStage ) From d32bb4e5ee4718741252c46c50a40810b722f12d Mon Sep 17 00:00:00 2001 From: Lena Date: Tue, 23 Mar 2021 10:13:32 +0300 Subject: [PATCH 03/24] [MINOR][DOCS] Updating the link for Azure Data Lake Gen 2 in docs ### What changes were proposed in this pull request? Current link for `Azure Blob Storage and Azure Datalake Gen 2` leads to AWS information. Replacing the link to point to the right page. ### Why are the changes needed? For users to access to the correct link. ### Does this PR introduce _any_ user-facing change? Yes, it fixes the link correctly. ### How was this patch tested? N/A Closes #31938 from lenadroid/patch-1. Authored-by: Lena Signed-off-by: Max Gekk --- docs/cloud-integration.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index d801df7254429..896ed77c87710 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -276,7 +276,8 @@ under-reported with Hadoop versions before 3.3.1. Here is the documentation on the standard connectors both from Apache and the cloud providers. * [OpenStack Swift](https://hadoop.apache.org/docs/current/hadoop-openstack/index.html). -* [Azure Blob Storage and Azure Datalake Gen 2](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). +* [Azure Blob Storage](https://hadoop.apache.org/docs/current/hadoop-azure/index.html). +* [Azure Blob Filesystem (ABFS) and Azure Datalake Gen 2](https://hadoop.apache.org/docs/current/hadoop-azure/abfs.html). * [Azure Data Lake Gen 1](https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). * [Amazon S3 Strong Consistency](https://aws.amazon.com/s3/consistency/) * [Hadoop-AWS module (Hadoop 3.x)](https://hadoop.apache.org/docs/current3/hadoop-aws/tools/hadoop-aws/index.html). From e00afd31a72350f388c4473704d5dd58f79ffa29 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 23 Mar 2021 07:38:43 +0000 Subject: [PATCH 04/24] [SPARK-34087][FOLLOW-UP][SQL] Manage ExecutionListenerBus register inside itself ### What changes were proposed in this pull request? Move `ExecutionListenerBus` register (both `ListenerBus` and `ContextCleaner` register) into itself. Also with a minor change that put `registerSparkListenerForCleanup` to a better place. ### Why are the changes needed? improve code ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass existing tests. Closes #31919 from Ngone51/SPARK-34087-followup. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/ContextCleaner.scala | 10 +++++----- .../spark/sql/util/QueryExecutionListener.scala | 16 ++++++++++------ 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 34b3089107efe..091b5e1600d9e 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -172,11 +172,6 @@ private[spark] class ContextCleaner( registerForCleanup(rdd, CleanCheckpoint(parentId)) } - /** Register an object for cleanup. */ - private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = { - referenceBuffer.add(new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue)) - } - /** Register a SparkListener to be cleaned up when its owner is garbage collected. */ def registerSparkListenerForCleanup( listenerOwner: AnyRef, @@ -184,6 +179,11 @@ private[spark] class ContextCleaner( registerForCleanup(listenerOwner, CleanSparkListener(listener)) } + /** Register an object for cleanup. */ + private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = { + referenceBuffer.add(new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue)) + } + /** Keep cleaning RDD, shuffle, and broadcast state. */ private def keepCleaning(): Unit = Utils.tryOrStopSparkContext(sc) { while (!stopped) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala index d8b630d5354e0..b1742078cc125 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala @@ -76,11 +76,7 @@ trait QueryExecutionListener { class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) extends Logging { - private val listenerBus = new ExecutionListenerBus(session.sessionUUID) - session.sparkContext.listenerBus.addToSharedQueue(listenerBus) - session.sparkContext.cleaner.foreach { cleaner => - cleaner.registerSparkListenerForCleanup(this, listenerBus) - } + private val listenerBus = new ExecutionListenerBus(this, session) if (loadExtensions) { val conf = session.sparkContext.conf @@ -128,9 +124,17 @@ class ExecutionListenerManager private[sql](session: SparkSession, loadExtension } } -private[sql] class ExecutionListenerBus(sessionUUID: String) +private[sql] class ExecutionListenerBus private(sessionUUID: String) extends SparkListener with ListenerBus[QueryExecutionListener, SparkListenerSQLExecutionEnd] { + def this(manager: ExecutionListenerManager, session: SparkSession) = { + this(session.sessionUUID) + session.sparkContext.listenerBus.addToSharedQueue(this) + session.sparkContext.cleaner.foreach { cleaner => + cleaner.registerSparkListenerForCleanup(manager, this) + } + } + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { case e: SparkListenerSQLExecutionEnd => postToAll(e) case _ => From 93a5d34f84c362110ef7d8853e59ce597faddad9 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Tue, 23 Mar 2021 17:01:16 +0800 Subject: [PATCH 05/24] [SPARK-33482][SPARK-34756][SQL] Fix FileScan equality check ### What changes were proposed in this pull request? This bug was introduced by SPARK-30428 at Apache Spark 3.0.0. This PR fixes `FileScan.equals()`. ### Why are the changes needed? - Without this fix `FileScan.equals` doesn't take `fileIndex` and `readSchema` into account. - Partition filters and data filters added to `FileScan` (in #27112 and #27157) caused that canonicalized form of some `BatchScanExec` nodes don't match and this prevents some reuse possibilities. ### Does this PR introduce _any_ user-facing change? Yes, before this fix incorrect reuse of `FileScan` and so `BatchScanExec` could have happed causing correctness issues. ### How was this patch tested? Added new UTs. Closes #31848 from peter-toth/SPARK-34756-fix-filescan-equality-check. Authored-by: Peter Toth Signed-off-by: Wenchen Fan --- .../apache/spark/sql/avro/AvroScanSuite.scala | 30 ++ .../execution/datasources/v2/FileScan.scala | 22 +- .../org/apache/spark/sql/FileScanSuite.scala | 374 ++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 24 ++ 4 files changed, 446 insertions(+), 4 deletions(-) create mode 100644 external/avro/src/test/scala/org/apache/spark/sql/avro/AvroScanSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/FileScanSuite.scala diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroScanSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroScanSuite.scala new file mode 100644 index 0000000000000..98a7190ba984e --- /dev/null +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroScanSuite.scala @@ -0,0 +1,30 @@ +/* + * 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.avro + +import org.apache.spark.sql.FileScanSuiteBase +import org.apache.spark.sql.v2.avro.AvroScan + +class AvroScanSuite extends FileScanSuiteBase { + val scanBuilders = Seq[(String, ScanBuilder, Seq[String])]( + ("AvroScan", + (s, fi, ds, rds, rps, f, o, pf, df) => AvroScan(s, fi, ds, rds, rps, o, f, pf, df), + Seq.empty)) + + run(scanBuilders) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index 363dd154b5fbb..ac63725b774d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -24,8 +24,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.internal.config.IO_WARNING_LARGEFILETHRESHOLD import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionSet} +import org.apache.spark.sql.catalyst.expressions.{AttributeSet, Expression, ExpressionSet} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.connector.read.{Batch, InputPartition, Scan, Statistics, SupportsReportStatistics} import org.apache.spark.sql.execution.PartitionedFileUtil import org.apache.spark.sql.execution.datasources._ @@ -84,11 +85,24 @@ trait FileScan extends Scan protected def seqToString(seq: Seq[Any]): String = seq.mkString("[", ", ", "]") + private lazy val (normalizedPartitionFilters, normalizedDataFilters) = { + val output = readSchema().toAttributes + val partitionFilterAttributes = AttributeSet(partitionFilters).map(a => a.name -> a).toMap + val dataFiltersAttributes = AttributeSet(dataFilters).map(a => a.name -> a).toMap + val normalizedPartitionFilters = ExpressionSet(partitionFilters.map( + QueryPlan.normalizeExpressions(_, + output.map(a => partitionFilterAttributes.getOrElse(a.name, a))))) + val normalizedDataFilters = ExpressionSet(dataFilters.map( + QueryPlan.normalizeExpressions(_, + output.map(a => dataFiltersAttributes.getOrElse(a.name, a))))) + (normalizedPartitionFilters, normalizedDataFilters) + } + override def equals(obj: Any): Boolean = obj match { case f: FileScan => - fileIndex == f.fileIndex && readSchema == f.readSchema - ExpressionSet(partitionFilters) == ExpressionSet(f.partitionFilters) && - ExpressionSet(dataFilters) == ExpressionSet(f.dataFilters) + fileIndex == f.fileIndex && readSchema == f.readSchema && + normalizedPartitionFilters == f.normalizedPartitionFilters && + normalizedDataFilters == f.normalizedDataFilters case _ => false } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileScanSuite.scala new file mode 100644 index 0000000000000..4e7fe8455ff93 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileScanSuite.scala @@ -0,0 +1,374 @@ +/* + * 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 + +import scala.collection.mutable + +import com.google.common.collect.ImmutableMap +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{And, Expression, IsNull, LessThan} +import org.apache.spark.sql.execution.datasources.{PartitioningAwareFileIndex, PartitionSpec} +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan +import org.apache.spark.sql.execution.datasources.v2.json.JsonScan +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.execution.datasources.v2.text.TextScan +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +trait FileScanSuiteBase extends SharedSparkSession { + private def newPartitioningAwareFileIndex() = { + new PartitioningAwareFileIndex(spark, Map.empty, None) { + override def partitionSpec(): PartitionSpec = { + PartitionSpec.emptySpec + } + + override protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = { + mutable.LinkedHashMap.empty + } + + override protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = { + Map.empty + } + + override def rootPaths: Seq[Path] = { + Seq.empty + } + + override def refresh(): Unit = {} + } + } + + type ScanBuilder = ( + SparkSession, + PartitioningAwareFileIndex, + StructType, + StructType, + StructType, + Array[Filter], + CaseInsensitiveStringMap, + Seq[Expression], + Seq[Expression]) => FileScan + + def run(scanBuilders: Seq[(String, ScanBuilder, Seq[String])]): Unit = { + val dataSchema = StructType.fromDDL("data INT, partition INT, other INT") + val dataSchemaNotEqual = StructType.fromDDL("data INT, partition INT, other INT, new INT") + val readDataSchema = StructType.fromDDL("data INT") + val readDataSchemaNotEqual = StructType.fromDDL("data INT, other INT") + val readPartitionSchema = StructType.fromDDL("partition INT") + val readPartitionSchemaNotEqual = StructType.fromDDL("partition INT, other INT") + val pushedFilters = + Array[Filter](sources.And(sources.IsNull("data"), sources.LessThan("data", 0))) + val pushedFiltersNotEqual = + Array[Filter](sources.And(sources.IsNull("data"), sources.LessThan("data", 1))) + val optionsMap = ImmutableMap.of("key", "value") + val options = new CaseInsensitiveStringMap(ImmutableMap.copyOf(optionsMap)) + val optionsNotEqual = + new CaseInsensitiveStringMap(ImmutableMap.copyOf(ImmutableMap.of("key2", "value2"))) + val partitionFilters = Seq(And(IsNull('data.int), LessThan('data.int, 0))) + val partitionFiltersNotEqual = Seq(And(IsNull('data.int), LessThan('data.int, 1))) + val dataFilters = Seq(And(IsNull('data.int), LessThan('data.int, 0))) + val dataFiltersNotEqual = Seq(And(IsNull('data.int), LessThan('data.int, 1))) + + scanBuilders.foreach { case (name, scanBuilder, exclusions) => + test(s"SPARK-33482: Test $name equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanEquals = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema.copy(), + readDataSchema.copy(), + readPartitionSchema.copy(), + pushedFilters.clone(), + new CaseInsensitiveStringMap(ImmutableMap.copyOf(optionsMap)), + Seq(partitionFilters: _*), + Seq(dataFilters: _*)) + + assert(scan === scanEquals) + } + + test(s"SPARK-33482: Test $name fileIndex not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val partitioningAwareFileIndexNotEqual = newPartitioningAwareFileIndex() + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndexNotEqual, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + if (!exclusions.contains("dataSchema")) { + test(s"SPARK-33482: Test $name dataSchema not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchemaNotEqual, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + } + + test(s"SPARK-33482: Test $name readDataSchema not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchemaNotEqual, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + test(s"SPARK-33482: Test $name readPartitionSchema not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchemaNotEqual, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + if (!exclusions.contains("pushedFilters")) { + test(s"SPARK-33482: Test $name pushedFilters not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFiltersNotEqual, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + } + + test(s"SPARK-33482: Test $name options not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + optionsNotEqual, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + test(s"SPARK-33482: Test $name partitionFilters not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFiltersNotEqual, + dataFilters) + assert(scan !== scanNotEqual) + } + + test(s"SPARK-33482: Test $name dataFilters not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFiltersNotEqual) + assert(scan !== scanNotEqual) + } + } + } +} + +class FileScanSuite extends FileScanSuiteBase { + val scanBuilders = Seq[(String, ScanBuilder, Seq[String])]( + ("ParquetScan", + (s, fi, ds, rds, rps, f, o, pf, df) => + ParquetScan(s, s.sessionState.newHadoopConf(), fi, ds, rds, rps, f, o, pf, df), + Seq.empty), + ("OrcScan", + (s, fi, ds, rds, rps, f, o, pf, df) => + OrcScan(s, s.sessionState.newHadoopConf(), fi, ds, rds, rps, o, f, pf, df), + Seq.empty), + ("CSVScan", + (s, fi, ds, rds, rps, f, o, pf, df) => CSVScan(s, fi, ds, rds, rps, o, f, pf, df), + Seq.empty), + ("JsonScan", + (s, fi, ds, rds, rps, f, o, pf, df) => JsonScan(s, fi, ds, rds, rps, o, f, pf, df), + Seq.empty), + ("TextScan", + (s, fi, _, rds, rps, _, o, pf, df) => TextScan(s, fi, rds, rps, o, pf, df), + Seq("dataSchema", "pushedFilters"))) + + run(scanBuilders) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 00cbd73533ab9..7e7853e1799d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.execution.datasources.{LogicalRelation, SchemaColumn import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -4116,6 +4117,29 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } } + + test("SPARK-33482: Fix FileScan canonicalization") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + withTempPath { path => + spark.range(5).toDF().write.mode("overwrite").parquet(path.toString) + withTempView("t") { + spark.read.parquet(path.toString).createOrReplaceTempView("t") + val df = sql( + """ + |SELECT * + |FROM t AS t1 + |JOIN t AS t2 ON t2.id = t1.id + |JOIN t AS t3 ON t3.id = t2.id + |""".stripMargin) + df.collect() + val reusedExchanges = collect(df.queryExecution.executedPlan) { + case r: ReusedExchangeExec => r + } + assert(reusedExchanges.size == 1) + } + } + } + } } case class Foo(bar: Option[String]) From 115ed89a3cd75faea3e6e29fb580da45309c0f31 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 23 Mar 2021 13:22:37 +0000 Subject: [PATCH 06/24] [SPARK-34366][SQL] Add interface for DS v2 metrics ### What changes were proposed in this pull request? This patch proposes to add a few public API change to DS v2, to make DS v2 scan can report metrics to Spark. Two public interfaces are added. * `CustomMetric`: metric interface at the driver side. It basically defines how Spark aggregates task metrics with the same metric name. * `CustomTaskMetric`: task metric reported at executors. It includes a name and long value. Spark will collect these metric values and update internal metrics. There are two public methods added to existing public interfaces. They are optional to DS v2 implementations. * `PartitionReader.currentMetricsValues()`: returns an array of CustomTaskMetric. Here is where the actual metrics values are collected. Empty array by default. * `Scan.supportedCustomMetrics()`: returns an array of supported custom metrics `CustomMetric`. Empty array by default. ### Why are the changes needed? In order to report custom metrics, we need some public API change in DS v2 to make it possible. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? This only adds interfaces. In follow-up PRs where adding implementation there will be tests added. See #31451 and #31398 for some details and manual test there. Closes #31476 from viirya/SPARK-34366. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../spark/sql/connector/CustomMetric.java | 51 +++++++++++++++++++ .../spark/sql/connector/CustomTaskMetric.java | 46 +++++++++++++++++ .../sql/connector/read/PartitionReader.java | 9 ++++ .../apache/spark/sql/connector/read/Scan.java | 10 ++++ 4 files changed, 116 insertions(+) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/CustomMetric.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/CustomTaskMetric.java diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/CustomMetric.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/CustomMetric.java new file mode 100644 index 0000000000000..bbd35ac946773 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/CustomMetric.java @@ -0,0 +1,51 @@ +/* + * 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.connector; + +import org.apache.spark.annotation.Evolving; + +/** + * A custom metric. Data source can define supported custom metrics using this interface. + * During query execution, Spark will collect the task metrics using {@link CustomTaskMetric} + * and combine the metrics at the driver side. How to combine task metrics is defined by the + * metric class with the same metric name. + * + * @since 3.2.0 + */ +@Evolving +public interface CustomMetric { + /** + * Returns the name of custom metric. + */ + String name(); + + /** + * Returns the description of custom metric. + */ + String description(); + + /** + * The initial value of this metric. + */ + long initialValue = 0L; + + /** + * Given an array of task metric values, returns aggregated final metric value. + */ + String aggregateTaskMetrics(long[] taskMetrics); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/CustomTaskMetric.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/CustomTaskMetric.java new file mode 100644 index 0000000000000..47644a3267edf --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/CustomTaskMetric.java @@ -0,0 +1,46 @@ +/* + * 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.connector; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.connector.read.PartitionReader; + +/** + * A custom task metric. This is a logical representation of a metric reported by data sources + * at the executor side. During query execution, Spark will collect the task metrics per partition + * by {@link PartitionReader} and update internal metrics based on collected metric values. + * For streaming query, Spark will collect and combine metrics for a final result per micro batch. + * + * The metrics will be gathered during query execution back to the driver and then combined. How + * the task metrics are combined is defined by corresponding {@link CustomMetric} with same metric + * name. The final result will be shown up in the data source scan operator in Spark UI. + * + * @since 3.2.0 + */ +@Evolving +public interface CustomTaskMetric { + /** + * Returns the name of custom task metric. + */ + String name(); + + /** + * Returns the long value of custom task metric. + */ + long value(); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java index 23fbd95800e26..dfecb77c669b7 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.connector.CustomTaskMetric; /** * A partition reader returned by {@link PartitionReaderFactory#createReader(InputPartition)} or @@ -48,4 +49,12 @@ public interface PartitionReader extends Closeable { * Return the current record. This method should return same value until `next` is called. */ T get(); + + /** + * Returns an array of custom task metrics. By default it returns empty array. + */ + default CustomTaskMetric[] currentMetricsValues() { + CustomTaskMetric[] NO_METRICS = {}; + return NO_METRICS; + } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Scan.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Scan.java index 4146f217985b4..b70a656c492a8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Scan.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Scan.java @@ -18,6 +18,7 @@ package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.connector.CustomMetric; import org.apache.spark.sql.connector.read.streaming.ContinuousStream; import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; import org.apache.spark.sql.types.StructType; @@ -102,4 +103,13 @@ default MicroBatchStream toMicroBatchStream(String checkpointLocation) { default ContinuousStream toContinuousStream(String checkpointLocation) { throw new UnsupportedOperationException(description() + ": Continuous scan are not supported"); } + + /** + * Returns an array of supported custom metrics with name and description. + * By default it returns empty array. + */ + default CustomMetric[] supportedCustomMetrics() { + CustomMetric[] NO_METRICS = {}; + return NO_METRICS; + } } From 3b70829b5b2944865c23daef4106f115312ab40a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 23 Mar 2021 14:34:51 +0000 Subject: [PATCH 07/24] [SPARK-34719][SQL] Correctly resolve the view query with duplicated column names forward-port https://github.com/apache/spark/pull/31811 to master ### What changes were proposed in this pull request? For permanent views (and the new SQL temp view in Spark 3.1), we store the view SQL text and re-parse/analyze the view SQL text when reading the view. In the case of `SELECT * FROM ...`, we want to avoid view schema change (e.g. the referenced table changes its schema) and will record the view query output column names when creating the view, so that when reading the view we can add a `SELECT recorded_column_names FROM ...` to retain the original view query schema. In Spark 3.1 and before, the final SELECT is added after the analysis phase: https://github.com/apache/spark/blob/branch-3.1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala#L67 If the view query has duplicated output column names, we always pick the first column when reading a view. A simple repro: ``` scala> sql("create view c(x, y) as select 1 a, 2 a") res0: org.apache.spark.sql.DataFrame = [] scala> sql("select * from c").show +---+---+ | x| y| +---+---+ | 1| 1| +---+---+ ``` In the master branch, we will fail at the view reading time due to https://github.com/apache/spark/commit/b891862fb6b740b103d5a09530626ee4e0e8f6e3 , which adds the final SELECT during analysis, so that the query fails with `Reference 'a' is ambiguous` This PR proposes to resolve the view query output column names from the matching attributes by ordinal. For example, `create view c(x, y) as select 1 a, 2 a`, the view query output column names are `[a, a]`. When we reading the view, there are 2 matching attributes (e.g.`[a#1, a#2]`) and we can simply match them by ordinal. A negative example is ``` create table t(a int) create view v as select *, 1 as col from t replace table t(a int, col int) ``` When reading the view, the view query output column names are `[a, col]`, and there are two matching attributes of `col`, and we should fail the query. See the tests for details. ### Why are the changes needed? bug fix ### Does this PR introduce _any_ user-facing change? yes ### How was this patch tested? new test Closes #31930 from cloud-fan/view2. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 30 ++++++++---- .../sql/catalyst/analysis/unresolved.scala | 11 +++++ .../sql/catalyst/catalog/SessionCatalog.scala | 24 +++++++++- .../sql/errors/QueryCompilationErrors.scala | 10 ++++ .../sql/execution/SQLViewTestSuite.scala | 48 +++++++++++++++++++ 5 files changed, 112 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 98663a8d807be..68a3f3d5aa4b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1889,13 +1889,27 @@ class Analyzer(override val catalogManager: CatalogManager) private def resolveExpression( expr: Expression, resolveColumnByName: Seq[String] => Option[Expression], - resolveColumnByOrdinal: Int => Attribute, + getAttrCandidates: () => Seq[Attribute], throws: Boolean): Expression = { def innerResolve(e: Expression, isTopLevel: Boolean): Expression = { if (e.resolved) return e e match { case f: LambdaFunction if !f.bound => f - case GetColumnByOrdinal(ordinal, _) => resolveColumnByOrdinal(ordinal) + + case GetColumnByOrdinal(ordinal, _) => + val attrCandidates = getAttrCandidates() + assert(ordinal >= 0 && ordinal < attrCandidates.length) + attrCandidates(ordinal) + + case GetViewColumnByNameAndOrdinal(viewName, colName, ordinal, expectedNumCandidates) => + val attrCandidates = getAttrCandidates() + val matched = attrCandidates.filter(a => resolver(a.name, colName)) + if (matched.length != expectedNumCandidates) { + throw QueryCompilationErrors.incompatibleViewSchemaChange( + viewName, colName, expectedNumCandidates, matched) + } + matched(ordinal) + case u @ UnresolvedAttribute(nameParts) => val result = withPosition(u) { resolveColumnByName(nameParts).orElse(resolveLiteralFunction(nameParts)).map { @@ -1909,6 +1923,7 @@ class Analyzer(override val catalogManager: CatalogManager) } logDebug(s"Resolving $u to $result") result + case u @ UnresolvedExtractValue(child, fieldName) => val newChild = innerResolve(child, isTopLevel = false) if (newChild.resolved) { @@ -1916,6 +1931,7 @@ class Analyzer(override val catalogManager: CatalogManager) } else { u.copy(child = newChild) } + case _ => e.mapChildren(innerResolve(_, isTopLevel = false)) } } @@ -1948,10 +1964,7 @@ class Analyzer(override val catalogManager: CatalogManager) resolveColumnByName = nameParts => { plan.resolve(nameParts, resolver) }, - resolveColumnByOrdinal = ordinal => { - assert(ordinal >= 0 && ordinal < plan.output.length) - plan.output(ordinal) - }, + getAttrCandidates = () => plan.output, throws = throws) } @@ -1971,10 +1984,9 @@ class Analyzer(override val catalogManager: CatalogManager) resolveColumnByName = nameParts => { q.resolveChildren(nameParts, resolver) }, - resolveColumnByOrdinal = ordinal => { + getAttrCandidates = () => { assert(q.children.length == 1) - assert(ordinal >= 0 && ordinal < q.children.head.output.length) - q.children.head.output(ordinal) + q.children.head.output }, throws = true) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index c7287c4b8b17c..a273669d8876b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -544,6 +544,17 @@ case class GetColumnByOrdinal(ordinal: Int, dataType: DataType) extends LeafExpr override lazy val resolved = false } +case class GetViewColumnByNameAndOrdinal( + viewName: String, + colName: String, + ordinal: Int, + expectedNumCandidates: Int) + extends LeafExpression with Unevaluable with NonSQLExpression { + override def dataType: DataType = throw new UnresolvedException("dataType") + override def nullable: Boolean = throw new UnresolvedException("nullable") + override lazy val resolved = false +} + /** * Represents unresolved ordinal used in order by or group by. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 41c74bfab44ec..9df5a03ebea80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -847,9 +847,29 @@ class SessionCatalog( // the extra columns that we don't require), with UpCast (to make sure the type change is // safe) and Alias (to respect user-specified view column names) according to the view schema // in the catalog. + // Note that, the column names may have duplication, e.g. `CREATE VIEW v(x, y) AS + // SELECT 1 col, 2 col`. We need to make sure that the matching attributes have the same + // number of duplications, and pick the corresponding attribute by ordinal. + val viewConf = View.effectiveSQLConf(metadata.viewSQLConfigs, isTempView) + val normalizeColName: String => String = if (viewConf.caseSensitiveAnalysis) { + identity + } else { + _.toLowerCase(Locale.ROOT) + } + val nameToCounts = viewColumnNames.groupBy(normalizeColName).mapValues(_.length) + val nameToCurrentOrdinal = scala.collection.mutable.HashMap.empty[String, Int] + val projectList = viewColumnNames.zip(metadata.schema).map { case (name, field) => - Alias(UpCast(UnresolvedAttribute.quoted(name), field.dataType), field.name)( - explicitMetadata = Some(field.metadata)) + val normalizedName = normalizeColName(name) + val count = nameToCounts(normalizedName) + val col = if (count > 1) { + val ordinal = nameToCurrentOrdinal.getOrElse(normalizedName, 0) + nameToCurrentOrdinal(normalizedName) = ordinal + 1 + GetViewColumnByNameAndOrdinal(metadata.identifier.toString, name, ordinal, count) + } else { + UnresolvedAttribute.quoted(name) + } + Alias(UpCast(col, field.dataType), field.name)(explicitMetadata = Some(field.metadata)) } View(desc = metadata, isTempView = isTempView, child = Project(projectList, parsedPlan)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index d314eca426448..7e890cc31f52e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1087,4 +1087,14 @@ private[spark] object QueryCompilationErrors { def cannotConvertDataTypeToParquetTypeError(field: StructField): Throwable = { new AnalysisException(s"Unsupported data type ${field.dataType.catalogString}") } + + def incompatibleViewSchemaChange( + viewName: String, + colName: String, + expectedNum: Int, + actualCols: Seq[Attribute]): Throwable = { + new AnalysisException(s"The SQL query of view $viewName has an incompatible schema change " + + s"and column $colName cannot be resolved. Expected $expectedNum columns named $colName but " + + s"got ${actualCols.map(_.name).mkString("[", ",", "]")}") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala index 5124d4e4a7041..5fb2e2cd3d433 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala @@ -327,6 +327,54 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { } } } + + test("SPARK-34719: view query with duplicated output column names") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(CASE_SENSITIVE.key -> caseSensitive.toString) { + withView("v1", "v2") { + sql("CREATE VIEW v1 AS SELECT 1 a, 2 b") + sql("CREATE VIEW v2 AS SELECT 1 col") + + val viewName = createView( + viewName = "testView", + sqlText = "SELECT *, 1 col, 2 col FROM v1", + columnNames = Seq("c1", "c2", "c3", "c4")) + withView(viewName) { + checkViewOutput(viewName, Seq(Row(1, 2, 1, 2))) + + // One more duplicated column `COL` if caseSensitive=false. + sql("CREATE OR REPLACE VIEW v1 AS SELECT 1 a, 2 b, 3 COL") + if (caseSensitive) { + checkViewOutput(viewName, Seq(Row(1, 2, 1, 2))) + } else { + val e = intercept[AnalysisException](spark.table(viewName).collect()) + assert(e.message.contains("incompatible schema change")) + } + } + + // v1 has 3 columns [a, b, COL], v2 has one column [col], so `testView2` has duplicated + // output column names if caseSensitive=false. + val viewName2 = createView( + viewName = "testView2", + sqlText = "SELECT * FROM v1, v2", + columnNames = Seq("c1", "c2", "c3", "c4")) + withView(viewName2) { + checkViewOutput(viewName2, Seq(Row(1, 2, 3, 1))) + + // One less duplicated column if caseSensitive=false. + sql("CREATE OR REPLACE VIEW v1 AS SELECT 1 a, 2 b") + if (caseSensitive) { + val e = intercept[AnalysisException](spark.table(viewName2).collect()) + assert(e.message.contains("cannot resolve 'COL'")) + } else { + val e = intercept[AnalysisException](spark.table(viewName2).collect()) + assert(e.message.contains("incompatible schema change")) + } + } + } + } + } + } } class LocalTempViewTestSuite extends SQLViewTestSuite with SharedSparkSession { From 06d40696dc27c39824666a70e7c673880959a681 Mon Sep 17 00:00:00 2001 From: robert4os Date: Wed, 24 Mar 2021 00:13:06 +0900 Subject: [PATCH 08/24] [MINOR][DOCS] Update sql-ref-syntax-dml-insert-into.md ### What changes were proposed in this pull request? the given example uses a non-standard syntax for CREATE TABLE, by defining the partitioning column with the other columns, instead of in PARTITION BY. This works is this case, because the partitioning column happens to be the last column defined, but it will break if instead 'name' would be used for partitioning. I suggest therefore to change the example to use a standard syntax, like in https://spark.apache.org/docs/3.1.1/sql-ref-syntax-ddl-create-table-hiveformat.html ### Why are the changes needed? To show the better documentation. ### Does this PR introduce _any_ user-facing change? Yes, this fixes the user-facing docs. ### How was this patch tested? CI should test it out. Closes #31900 from robert4os/patch-1. Authored-by: robert4os Signed-off-by: HyukjinKwon --- docs/sql-ref-syntax-dml-insert-into.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-syntax-dml-insert-into.md b/docs/sql-ref-syntax-dml-insert-into.md index 7c5904db225a5..1120c6e0a89df 100644 --- a/docs/sql-ref-syntax-dml-insert-into.md +++ b/docs/sql-ref-syntax-dml-insert-into.md @@ -70,8 +70,8 @@ INSERT INTO [ TABLE ] table_identifier [ partition_spec ] [ ( column_list ) ] #### Single Row Insert Using a VALUES Clause ```sql -CREATE TABLE students (name VARCHAR(64), address VARCHAR(64), student_id INT) - USING PARQUET PARTITIONED BY (student_id); +CREATE TABLE students (name VARCHAR(64), address VARCHAR(64)) + USING PARQUET PARTITIONED BY (student_id INT); INSERT INTO students VALUES ('Amy Smith', '123 Park Ave, San Jose', 111111); From 760556a42ff76509dfcd6f7e76712b31f878f466 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 23 Mar 2021 19:40:15 +0300 Subject: [PATCH 09/24] [SPARK-34824][SQL] Support multiply an year-month interval by a numeric ### What changes were proposed in this pull request? 1. Add new expression `MultiplyYMInterval` which multiplies a `YearMonthIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType. 2. Extend binary arithmetic rules to support `numeric * year-month interval` and `year-month interval * numeric`. ### Why are the changes needed? To conform the ANSI SQL standard which requires such operation over year-month intervals: Screenshot 2021-03-22 at 16 33 16 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running new tests: ``` $ build/sbt "test:testOnly *IntervalExpressionsSuite" $ build/sbt "test:testOnly *ColumnExpressionSuite" ``` Closes #31929 from MaxGekk/interval-mul-div. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../sql/catalyst/analysis/Analyzer.scala | 2 + .../expressions/intervalExpressions.scala | 46 +++++++++++++++++++ .../IntervalExpressionsSuite.scala | 37 ++++++++++++++- .../spark/sql/ColumnExpressionSuite.scala | 39 ++++++++++++++++ 4 files changed, 123 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 68a3f3d5aa4b3..947aeb56f6ca2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -372,6 +372,8 @@ class Analyzer(override val catalogManager: CatalogManager) case m @ Multiply(l, r, f) if m.childrenResolved => (l.dataType, r.dataType) match { case (CalendarIntervalType, _) => MultiplyInterval(l, r, f) case (_, CalendarIntervalType) => MultiplyInterval(r, l, f) + case (YearMonthIntervalType, _) => MultiplyYMInterval(l, r) + case (_, YearMonthIntervalType) => MultiplyYMInterval(r, l) case _ => m } case d @ Divide(l, r, f) if d.childrenResolved => (l.dataType, r.dataType) match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index fd07aff867abf..ea4bf1e40bdac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -249,3 +249,49 @@ case class MakeInterval( override def prettyName: String = "make_interval" } + +// Multiply an year-month interval by a numeric +case class MultiplyYMInterval( + interval: Expression, + num: Expression) + extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable { + override def left: Expression = interval + override def right: Expression = num + + override def inputTypes: Seq[AbstractDataType] = Seq(YearMonthIntervalType, NumericType) + override def dataType: DataType = YearMonthIntervalType + + @transient + private lazy val evalFunc: (Int, Any) => Any = right.dataType match { + case ByteType | ShortType | IntegerType => (months: Int, num) => + Math.multiplyExact(months, num.asInstanceOf[Number].intValue()) + case LongType => (months: Int, num) => + Math.toIntExact(Math.multiplyExact(months, num.asInstanceOf[Long])) + case FloatType | DoubleType => (months: Int, num) => + Math.toIntExact(Math.round(months * num.asInstanceOf[Number].doubleValue())) + case _: DecimalType => (months: Int, num) => + val decimalRes = ((new Decimal).set(months) * num.asInstanceOf[Decimal]).toJavaBigDecimal + decimalRes.setScale(0, java.math.RoundingMode.HALF_UP).intValueExact() + } + + override def nullSafeEval(interval: Any, num: Any): Any = { + evalFunc(interval.asInstanceOf[Int], num) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = right.dataType match { + case ByteType | ShortType | IntegerType => + defineCodeGen(ctx, ev, (m, n) => s"java.lang.Math.multiplyExact($m, $n)") + case LongType => + val jlm = classOf[Math].getName + defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.multiplyExact($m, $n))") + case FloatType | DoubleType => + val jlm = classOf[Math].getName + defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.round($m * (double)$n))") + case _: DecimalType => + defineCodeGen(ctx, ev, (m, n) => + s"((new Decimal()).set($m).$$times($n)).toJavaBigDecimal()" + + ".setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()") + } + + override def toString: String = s"($left * $right)" +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala index 950637c958426..f7f75f546d14e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.time.Period + import scala.language.implicitConversions import org.apache.spark.SparkFunSuite @@ -24,7 +26,8 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.catalyst.util.IntervalUtils.{safeStringToInterval, stringToInterval} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.Decimal +import org.apache.spark.sql.types.{Decimal, DecimalType, YearMonthIntervalType} +import org.apache.spark.sql.types.DataTypeTestUtils.numericTypes import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -274,4 +277,36 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkException(weeks = Int.MaxValue) } } + + test("SPARK-34824: multiply year-month interval by numeric") { + Seq( + (Period.ofMonths(-100), Float.NaN) -> Period.ofMonths(0), + (Period.ofYears(-123), Literal(null, DecimalType.USER_DEFAULT)) -> null, + (Period.ofMonths(0), 10) -> Period.ofMonths(0), + (Period.ofMonths(10), 0L) -> Period.ofMonths(0), + (Period.ofYears(100), -1.toByte) -> Period.ofYears(-100), + (Period.ofMonths(12), 0.3f) -> Period.ofMonths(4), + (Period.ofYears(-1000), 0.3d) -> Period.ofYears(-300), + (Period.ofYears(9999), 0.0001d) -> Period.ofYears(1), + (Period.ofYears(9999), BigDecimal(0.0001)) -> Period.ofYears(1) + ).foreach { case ((period, num), expected) => + checkEvaluation(MultiplyYMInterval(Literal(period), Literal(num)), expected) + } + + Seq( + (Period.ofMonths(2), Int.MaxValue), + (Period.ofMonths(Int.MinValue), 10d), + (Period.ofMonths(200), Double.PositiveInfinity), + (Period.ofMonths(-200), Float.NegativeInfinity) + ).foreach { case (period, num) => + checkExceptionInExpression[ArithmeticException]( + MultiplyYMInterval(Literal(period), Literal(num)), "overflow") + } + + numericTypes.foreach { numType => + checkConsistencyBetweenInterpretedAndCodegenAllowingException( + (interval: Expression, num: Expression) => MultiplyYMInterval(interval, num), + YearMonthIntervalType, numType) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 6ed64ece9b296..1d2a436a69792 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -2574,4 +2574,43 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } } } + + test("SPARK-34824: multiply year-month interval by numeric") { + checkAnswer( + Seq((Period.ofYears(0), 0)).toDF("i", "n").select($"i" * $"n"), + Row(Period.ofYears(0))) + checkAnswer( + Seq((Period.ofMonths(0), 10.toByte)).toDF("i", "n").select($"i" * $"n"), + Row(Period.ofMonths(0))) + checkAnswer( + Seq((Period.ofMonths(5), 3.toShort)).toDF("i", "n").select($"n" * $"i"), + Row(Period.ofYears(1).plusMonths(3))) + checkAnswer( + Seq((Period.ofYears(1000), "2")).toDF("i", "n").select($"i" * $"n"), + Row(Period.ofYears(2000))) + checkAnswer( + Seq((Period.ofMonths(1), 12L)).toDF("i", "n").select($"n" * $"i"), + Row(Period.ofYears(1))) + checkAnswer( + Seq((Period.ofYears(100).plusMonths(11), Short.MaxValue)).toDF("i", "n").select($"n" * $"i"), + Row(Period.ofYears(100).plusMonths(11).multipliedBy(Short.MaxValue).normalized())) + checkAnswer( + Seq((Period.ofMonths(-1), 0.499f)).toDF("i", "n").select($"i" * $"n"), + Row(Period.ofMonths(0))) + checkAnswer( + Seq((Period.ofMonths(10000000), 0.0000001d)).toDF("i", "n").select($"i" * $"n"), + Row(Period.ofMonths(1))) + checkAnswer( + Seq((Period.ofMonths(-10000000), BigDecimal(0.0000001d))).toDF("i", "n").select($"i" * $"n"), + Row(Period.ofMonths(-1))) + checkAnswer( + Seq((Period.ofMonths(-1), BigDecimal(0.5))).toDF("i", "n").select($"i" * $"n"), + Row(Period.ofMonths(-1))) + + val e = intercept[SparkException] { + Seq((Period.ofYears(9999), Long.MinValue)).toDF("i", "n").select($"n" * $"i").collect() + }.getCause + assert(e.isInstanceOf[ArithmeticException]) + assert(e.getMessage.contains("overflow")) + } } From 0494dc90af48ce7da0625485a4dc6917a244d580 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 23 Mar 2021 10:22:13 -0700 Subject: [PATCH 10/24] [SPARK-34842][SQL][TESTS] Corrects the type of `date_dim.d_quarter_name` in the TPCDS schema ### What changes were proposed in this pull request? SPARK-34842 (#31012) has a typo in the type of `date_dim.d_quarter_name` in the TPCDS schema (`TPCDSBase`). This PR replace `CHAR(1)` with `CHAR(6)`. This fix comes from p28 in [the TPCDS official doc](http://www.tpc.org/tpc_documents_current_versions/pdf/tpc-ds_v2.9.0.pdf). ### Why are the changes needed? Bugfix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? N/A Closes #31943 from maropu/SPARK-34083-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala index 3916fb9e3bc2f..78e4002236da9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala @@ -442,7 +442,7 @@ trait TPCDSBase extends SharedSparkSession { |`d_fy_quarter_seq` INT, |`d_fy_week_seq` INT, |`d_day_name` CHAR(9), - |`d_quarter_name` CHAR(1), + |`d_quarter_name` CHAR(6), |`d_holiday` CHAR(1), |`d_weekend` CHAR(1), |`d_following_holiday` CHAR(1), From 985c653b20a26907f89a753393c894cc5ff30229 Mon Sep 17 00:00:00 2001 From: hongdongdong Date: Tue, 23 Mar 2021 22:07:27 -0700 Subject: [PATCH 11/24] [SPARK-33720][K8S] Support submit to k8s only with token ### What changes were proposed in this pull request? Support submit to k8s only with token. ### Why are the changes needed? Now, sumbit to k8s always need oauth files. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Before, submit job out of k8s cluster without correct ca.crt, we may get this exception: ``` Caused by: sun.security.validator.ValidatorException: PKIX path building failed: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:439) at sun.security.validator.PKIXValidator.engineValidate(PKIXValidator.java:306) at sun.security.validator.Validator.validate(Validator.java:271) at sun.security.ssl.X509TrustManagerImpl.validate(X509TrustManagerImpl.java:312) ``` When set spark.kubernetes.trust.certificates = true, we can submit only with correct token, no need to config ca.crt in local env. Submit as: ``` bin/spark-submit \ --master $master \ --name pi \ --deploy-mode cluster \ --conf spark.kubernetes.container.image=$image \ --conf spark.kubernetes.authenticate.driver.serviceAccountName=spark \ --conf spark.kubernetes.authenticate.submission.oauthToken=$clusterToken \ --conf spark.kubernetes.trust.certificates=true \ local:///opt/spark/examples/src/main/python/pi.py 200 ``` Closes #30684 from hddong/trust-certs. Authored-by: hongdongdong Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/deploy/k8s/Config.scala | 7 +++++++ .../spark/deploy/k8s/SparkKubernetesClientFactory.scala | 1 + 2 files changed, 8 insertions(+) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 8ac2ee4792e17..36a68467019b2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -443,6 +443,13 @@ private[spark] object Config extends Logging { val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" + val KUBERNETES_TRUST_CERTIFICATES = + ConfigBuilder("spark.kubernetes.trust.certificates") + .doc("If set to true then client can submit to kubernetes cluster only with token") + .version("3.2.0") + .booleanConf + .createWithDefault(false) + val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." val KUBERNETES_DELETE_EXECUTORS = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index 459259f77796c..15b44474445a4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -83,6 +83,7 @@ private[spark] object SparkKubernetesClientFactory extends Logging { .withWebsocketPingInterval(0) .withRequestTimeout(clientType.requestTimeout(sparkConf)) .withConnectionTimeout(clientType.connectionTimeout(sparkConf)) + .withTrustCerts(sparkConf.get(KUBERNETES_TRUST_CERTIFICATES)) .withOption(oauthTokenValue) { (token, configBuilder) => configBuilder.withOauthToken(token) }.withOption(oauthTokenFile) { From f7e9b6efc70b6874587c440725d3af8efa3a316e Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 24 Mar 2021 13:34:10 +0800 Subject: [PATCH 12/24] [SPARK-34763][SQL] col(), $"" and df("name") should handle quoted column names properly ### What changes were proposed in this pull request? This PR fixes an issue that `col()`, `$""` and `df("name")` don't handle quoted column names like ``` `a``b.c` ```properly. For example, if we have a following DataFrame. ``` val df1 = spark.sql("SELECT 'col1' AS `a``b.c`") ``` For the DataFrame, this query is successfully executed. ``` scala> df1.selectExpr("`a``b.c`").show +-----+ |a`b.c| +-----+ | col1| +-----+ ``` But the following query will fail because ``` df1("`a``b.c`") ``` throws an exception. ``` scala> df1.select(df1("`a``b.c`")).show org.apache.spark.sql.AnalysisException: syntax error in attribute name: `a``b.c`; at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.e$1(unresolved.scala:152) at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.parseAttributeName(unresolved.scala:162) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveQuoted(LogicalPlan.scala:121) at org.apache.spark.sql.Dataset.resolve(Dataset.scala:221) at org.apache.spark.sql.Dataset.col(Dataset.scala:1274) at org.apache.spark.sql.Dataset.apply(Dataset.scala:1241) ... 49 elided ``` ### Why are the changes needed? It's a bug. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New tests. Closes #31854 from sarutak/fix-parseAttributeName. Authored-by: Kousuke Saruta Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/unresolved.scala | 11 +++++--- .../org/apache/spark/sql/DataFrameSuite.scala | 25 +++++++++++++++++++ 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index a273669d8876b..3fc3db30fa7b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -196,7 +196,7 @@ object UnresolvedAttribute { * Used to split attribute name by dot with backticks rule. * Backticks must appear in pairs, and the quoted string must be a complete name part, * which means `ab..c`e.f is not allowed. - * Escape character is not supported now, so we can't use backtick inside name part. + * We can use backtick only inside quoted name parts. */ def parseAttributeName(name: String): Seq[String] = { def e = QueryCompilationErrors.attributeNameSyntaxError(name) @@ -208,8 +208,13 @@ object UnresolvedAttribute { val char = name(i) if (inBacktick) { if (char == '`') { - inBacktick = false - if (i + 1 < name.length && name(i + 1) != '.') throw e + if (i + 1 < name.length && name(i + 1) == '`') { + tmp += '`' + i += 1 + } else { + inBacktick = false + if (i + 1 < name.length && name(i + 1) != '.') throw e + } } else { tmp += char } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 6e279f3f05c68..3e0312d11d92e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -2738,6 +2738,31 @@ class DataFrameSuite extends QueryTest checkAnswer(df.select(col2), Row(1) :: Row(2) :: Row(3) :: Nil) } + test("SPARK-34763: col(), $\"\", df(\"name\") should handle quoted column name properly") { + val df1 = spark.sql("SELECT 'col1' AS `a``b.c`") + checkAnswer(df1.selectExpr("`a``b.c`"), Row("col1")) + checkAnswer(df1.select(df1("`a``b.c`")), Row("col1")) + checkAnswer(df1.select(col("`a``b.c`")), Row("col1")) + checkAnswer(df1.select($"`a``b.c`"), Row("col1")) + + val df2 = df1.as("d.e`f") + checkAnswer(df2.selectExpr("`a``b.c`"), Row("col1")) + checkAnswer(df2.select(df2("`a``b.c`")), Row("col1")) + checkAnswer(df2.select(col("`a``b.c`")), Row("col1")) + checkAnswer(df2.select($"`a``b.c`"), Row("col1")) + + checkAnswer(df2.selectExpr("`d.e``f`.`a``b.c`"), Row("col1")) + checkAnswer(df2.select(df2("`d.e``f`.`a``b.c`")), Row("col1")) + checkAnswer(df2.select(col("`d.e``f`.`a``b.c`")), Row("col1")) + checkAnswer(df2.select($"`d.e``f`.`a``b.c`"), Row("col1")) + + val df3 = df1.as("*-#&% ?") + checkAnswer(df3.selectExpr("`*-#&% ?`.`a``b.c`"), Row("col1")) + checkAnswer(df3.select(df3("*-#&% ?.`a``b.c`")), Row("col1")) + checkAnswer(df3.select(col("*-#&% ?.`a``b.c`")), Row("col1")) + checkAnswer(df3.select($"*-#&% ?.`a``b.c`"), Row("col1")) + } + test("SPARK-34776: Nested column pruning should not prune Window produced attributes") { val df = Seq( ("t1", "123", "bob"), From 712a62ca8259539a76f45d9a54ccac8857b12a81 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Wed, 24 Mar 2021 14:59:31 +0900 Subject: [PATCH 13/24] [SPARK-34832][SQL][TEST] Set EXECUTOR_ALLOW_SPARK_CONTEXT to true to ensure ExternalAppendOnlyUnsafeRowArrayBenchmark run successfully MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? SPARK-32160 add a config(`EXECUTOR_ALLOW_SPARK_CONTEXT`) to switch allow/disallow to create `SparkContext` in executors and the default value of the config is `false` `ExternalAppendOnlyUnsafeRowArrayBenchmark` will run fail when `EXECUTOR_ALLOW_SPARK_CONTEXT` use the default value because the `ExternalAppendOnlyUnsafeRowArrayBenchmark#withFakeTaskContext` method try to create a `SparkContext` manually in Executor Side. So the main change of this pr is set `EXECUTOR_ALLOW_SPARK_CONTEXT` to `true` to ensure `ExternalAppendOnlyUnsafeRowArrayBenchmark` run successfully. ### Why are the changes needed? Bug fix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual test: ``` bin/spark-submit --class org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark --jars spark-core_2.12-3.2.0-SNAPSHOT-tests.jar spark-sql_2.12-3.2.0-SNAPSHOT-tests.jar ``` **Before** ``` Exception in thread "main" java.lang.IllegalStateException: SparkContext should only be created and accessed on the driver. at org.apache.spark.SparkContext$.org$apache$spark$SparkContext$$assertOnDriver(SparkContext.scala:2679) at org.apache.spark.SparkContext.(SparkContext.scala:89) at org.apache.spark.SparkContext.(SparkContext.scala:137) at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.withFakeTaskContext(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:52) at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.testAgainstRawArrayBuffer(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:119) at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.$anonfun$runBenchmarkSuite$1(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:189) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.apache.spark.benchmark.BenchmarkBase.runBenchmark(BenchmarkBase.scala:40) at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.runBenchmarkSuite(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:186) at org.apache.spark.benchmark.BenchmarkBase.main(BenchmarkBase.scala:58) at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark.main(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) ``` **After** `ExternalAppendOnlyUnsafeRowArrayBenchmark` run successfully. Closes #31939 from LuciferYang/SPARK-34832. Authored-by: yangjie01 Signed-off-by: HyukjinKwon --- .../execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala index 0869e25674e69..8962e923cccfb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala @@ -47,6 +47,9 @@ object ExternalAppendOnlyUnsafeRowArrayBenchmark extends BenchmarkBase { // for a bug we had with bytes written past the last object in a batch (SPARK-2792) .set(config.SERIALIZER_OBJECT_STREAM_RESET, 1) .set(config.SERIALIZER, "org.apache.spark.serializer.JavaSerializer") + // SPARK-34832: Add this configuration to allow `withFakeTaskContext` method + // to create `SparkContext` on the executor side. + .set(config.EXECUTOR_ALLOW_SPARK_CONTEXT, true) private def withFakeTaskContext(f: => Unit): Unit = { val sc = new SparkContext("local", "test", conf) From 2298cebcf83f7cad21fa291a718505032830474a Mon Sep 17 00:00:00 2001 From: Shardul Mahadik Date: Wed, 24 Mar 2021 15:31:34 +0800 Subject: [PATCH 14/24] [SPARK-34477][CORE] Register KryoSerializers for Avro GenericData classes ### What changes were proposed in this pull request? 1) Modify `GenericAvroSerializer` to support serialization of any `GenericContainer` 2) Register `KryoSerializer`s for `GenericData.{Array, EnumSymbol, Fixed}` using the modified `GenericAvroSerializer` ### Why are the changes needed? Without this change, Kryo throws NPEs when trying to serialize `GenericData.{Array, EnumSymbol, Fixed}`. More details in SPARK-34477 Jira ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added unit tests for testing roundtrip serialization and deserialization of `GenericData.{Array, EnumSymbol, Fixed}` using `GenericAvroSerializer` directly and also indirectly through `KryoSerializer` Closes #31597 from shardulm94/avro-array-serializer. Authored-by: Shardul Mahadik Signed-off-by: Gengliang Wang --- .../serializer/GenericAvroSerializer.scala | 29 +++---- .../spark/serializer/KryoSerializer.scala | 16 +++- .../GenericAvroSerializerSuite.scala | 78 +++++++++++++------ 3 files changed, 81 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala index f0ed41f6903f4..c1ef3ee769af2 100644 --- a/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala @@ -26,7 +26,7 @@ import scala.collection.mutable import com.esotericsoftware.kryo.{Kryo, Serializer => KSerializer} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import org.apache.avro.{Schema, SchemaNormalization} -import org.apache.avro.generic.{GenericData, GenericRecord} +import org.apache.avro.generic.{GenericContainer, GenericData} import org.apache.avro.io._ import org.apache.commons.io.IOUtils @@ -35,7 +35,7 @@ import org.apache.spark.io.CompressionCodec import org.apache.spark.util.Utils /** - * Custom serializer used for generic Avro records. If the user registers the schemas + * Custom serializer used for generic Avro containers. If the user registers the schemas * ahead of time, then the schema's fingerprint will be sent with each message instead of the actual * schema, as to reduce network IO. * Actions like parsing or compressing schemas are computationally expensive so the serializer @@ -43,9 +43,10 @@ import org.apache.spark.util.Utils * @param schemas a map where the keys are unique IDs for Avro schemas and the values are the * string representation of the Avro schema, used to decrease the amount of data * that needs to be serialized. + * @tparam D the subtype of [[GenericContainer]] handled by this serializer */ -private[serializer] class GenericAvroSerializer(schemas: Map[Long, String]) - extends KSerializer[GenericRecord] { +private[serializer] class GenericAvroSerializer[D <: GenericContainer] + (schemas: Map[Long, String]) extends KSerializer[D] { /** Used to reduce the amount of effort to compress the schema */ private val compressCache = new mutable.HashMap[Schema, Array[Byte]]() @@ -100,10 +101,10 @@ private[serializer] class GenericAvroSerializer(schemas: Map[Long, String]) }) /** - * Serializes a record to the given output stream. It caches a lot of the internal data as - * to not redo work + * Serializes a generic container to the given output stream. It caches a lot of the internal + * data as to not redo work */ - def serializeDatum[R <: GenericRecord](datum: R, output: KryoOutput): Unit = { + def serializeDatum(datum: D, output: KryoOutput): Unit = { val encoder = EncoderFactory.get.binaryEncoder(output, null) val schema = datum.getSchema val fingerprint = fingerprintCache.getOrElseUpdate(schema, { @@ -121,16 +122,16 @@ private[serializer] class GenericAvroSerializer(schemas: Map[Long, String]) } writerCache.getOrElseUpdate(schema, GenericData.get.createDatumWriter(schema)) - .asInstanceOf[DatumWriter[R]] + .asInstanceOf[DatumWriter[D]] .write(datum, encoder) encoder.flush() } /** - * Deserializes generic records into their in-memory form. There is internal + * Deserializes generic containers into their in-memory form. There is internal * state to keep a cache of already seen schemas and datum readers. */ - def deserializeDatum(input: KryoInput): GenericRecord = { + def deserializeDatum(input: KryoInput): D = { val schema = { if (input.readBoolean()) { val fingerprint = input.readLong() @@ -151,13 +152,13 @@ private[serializer] class GenericAvroSerializer(schemas: Map[Long, String]) } val decoder = DecoderFactory.get.directBinaryDecoder(input, null) readerCache.getOrElseUpdate(schema, GenericData.get.createDatumReader(schema)) - .asInstanceOf[DatumReader[GenericRecord]] - .read(null, decoder) + .asInstanceOf[DatumReader[D]] + .read(null.asInstanceOf[D], decoder) } - override def write(kryo: Kryo, output: KryoOutput, datum: GenericRecord): Unit = + override def write(kryo: Kryo, output: KryoOutput, datum: D): Unit = serializeDatum(datum, output) - override def read(kryo: Kryo, input: KryoInput, datumClass: Class[GenericRecord]): GenericRecord = + override def read(kryo: Kryo, input: KryoInput, datumClass: Class[D]): D = deserializeDatum(input) } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 55ac2c410953b..ce45762d20eab 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -33,7 +33,7 @@ import com.esotericsoftware.kryo.io.{UnsafeInput => KryoUnsafeInput, UnsafeOutpu import com.esotericsoftware.kryo.pool.{KryoCallback, KryoFactory, KryoPool} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} -import org.apache.avro.generic.{GenericData, GenericRecord} +import org.apache.avro.generic.{GenericContainer, GenericData, GenericRecord} import org.roaringbitmap.RoaringBitmap import org.apache.spark._ @@ -153,8 +153,18 @@ class KryoSerializer(conf: SparkConf) kryo.register(classOf[SerializableJobConf], new KryoJavaSerializer()) kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer()) - kryo.register(classOf[GenericRecord], new GenericAvroSerializer(avroSchemas)) - kryo.register(classOf[GenericData.Record], new GenericAvroSerializer(avroSchemas)) + // Register serializers for Avro GenericContainer classes + // We do not handle SpecificRecordBase and SpecificFixed here. They are abstract classes and + // we will need to register serializers for their concrete implementations individually. + // Also, their serialization requires the use of SpecificDatum(Reader|Writer) instead of + // GenericDatum(Reader|Writer). + def registerAvro[T <: GenericContainer]()(implicit ct: ClassTag[T]): Unit = + kryo.register(ct.runtimeClass, new GenericAvroSerializer[T](avroSchemas)) + registerAvro[GenericRecord] + registerAvro[GenericData.Record] + registerAvro[GenericData.Array[_]] + registerAvro[GenericData.EnumSymbol] + registerAvro[GenericData.Fixed] // Use the default classloader when calling the user registrator. Utils.withContextClassLoader(classLoader) { diff --git a/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala index 8610b18702ec0..54e4aebe54430 100644 --- a/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala @@ -22,53 +22,55 @@ import java.nio.ByteBuffer import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.avro.{Schema, SchemaBuilder} -import org.apache.avro.generic.GenericData.Record +import org.apache.avro.generic.GenericData.{Array => AvroArray, EnumSymbol, Fixed, Record} import org.apache.spark.{SharedSparkContext, SparkFunSuite} import org.apache.spark.internal.config.SERIALIZER class GenericAvroSerializerSuite extends SparkFunSuite with SharedSparkContext { - conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") - val schema : Schema = SchemaBuilder + override def beforeAll(): Unit = { + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + super.beforeAll() + } + + val recordSchema : Schema = SchemaBuilder .record("testRecord").fields() .requiredString("data") .endRecord() - val record = new Record(schema) - record.put("data", "test data") + val recordDatum = new Record(recordSchema) + recordDatum.put("data", "test data") - test("schema compression and decompression") { - val genericSer = new GenericAvroSerializer(conf.getAvroSchema) - assert(schema === genericSer.decompress(ByteBuffer.wrap(genericSer.compress(schema)))) - } + val arraySchema = SchemaBuilder.array().items().`type`(recordSchema) + val arrayDatum = new AvroArray[Record](1, arraySchema) + arrayDatum.add(recordDatum) - test("record serialization and deserialization") { - val genericSer = new GenericAvroSerializer(conf.getAvroSchema) + val enumSchema = SchemaBuilder.enumeration("enum").symbols("A", "B") + val enumDatum = new EnumSymbol(enumSchema, "A") - val outputStream = new ByteArrayOutputStream() - val output = new Output(outputStream) - genericSer.serializeDatum(record, output) - output.flush() - output.close() + val fixedSchema = SchemaBuilder.fixed("fixed").size(4) + val fixedDatum = new Fixed(fixedSchema, "ABCD".getBytes) - val input = new Input(new ByteArrayInputStream(outputStream.toByteArray)) - assert(genericSer.deserializeDatum(input) === record) + test("schema compression and decompression") { + val genericSer = new GenericAvroSerializer(conf.getAvroSchema) + assert(recordSchema === + genericSer.decompress(ByteBuffer.wrap(genericSer.compress(recordSchema)))) } test("uses schema fingerprint to decrease message size") { - val genericSerFull = new GenericAvroSerializer(conf.getAvroSchema) + val genericSerFull = new GenericAvroSerializer[Record](conf.getAvroSchema) val output = new Output(new ByteArrayOutputStream()) val beginningNormalPosition = output.total() - genericSerFull.serializeDatum(record, output) + genericSerFull.serializeDatum(recordDatum, output) output.flush() val normalLength = output.total - beginningNormalPosition - conf.registerAvroSchemas(schema) - val genericSerFinger = new GenericAvroSerializer(conf.getAvroSchema) + conf.registerAvroSchemas(recordSchema) + val genericSerFinger = new GenericAvroSerializer[Record](conf.getAvroSchema) val beginningFingerprintPosition = output.total() - genericSerFinger.serializeDatum(record, output) + genericSerFinger.serializeDatum(recordDatum, output) val fingerprintLength = output.total - beginningFingerprintPosition assert(fingerprintLength < normalLength) @@ -76,10 +78,36 @@ class GenericAvroSerializerSuite extends SparkFunSuite with SharedSparkContext { test("caches previously seen schemas") { val genericSer = new GenericAvroSerializer(conf.getAvroSchema) - val compressedSchema = genericSer.compress(schema) + val compressedSchema = genericSer.compress(recordSchema) val decompressedSchema = genericSer.decompress(ByteBuffer.wrap(compressedSchema)) - assert(compressedSchema.eq(genericSer.compress(schema))) + assert(compressedSchema.eq(genericSer.compress(recordSchema))) assert(decompressedSchema.eq(genericSer.decompress(ByteBuffer.wrap(compressedSchema)))) } + + Seq( + ("Record", recordDatum), + ("Array", arrayDatum), + ("EnumSymbol", enumDatum), + ("Fixed", fixedDatum) + ).foreach { case (name, datum) => + test(s"SPARK-34477: GenericData.$name serialization and deserialization") { + val genericSer = new GenericAvroSerializer[datum.type](conf.getAvroSchema) + + val outputStream = new ByteArrayOutputStream() + val output = new Output(outputStream) + genericSer.serializeDatum(datum, output) + output.flush() + output.close() + + val input = new Input(new ByteArrayInputStream(outputStream.toByteArray)) + assert(genericSer.deserializeDatum(input) === datum) + } + + test(s"SPARK-34477: GenericData.$name serialization and deserialization" + + " through KryoSerializer ") { + val rdd = sc.parallelize((0 until 10).map(_ => datum), 2) + assert(rdd.collect() sameElements Array.fill(10)(datum)) + } + } } From 95c61df0faed325b4d6912e3ca7c90e51a2a7eac Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 24 Mar 2021 01:11:53 -0700 Subject: [PATCH 15/24] [SPARK-34295][CORE] Exclude filesystems from token renewal at YARN ### What changes were proposed in this pull request? This patch adds a config `spark.yarn.kerberos.renewal.excludeHadoopFileSystems` which lists the filesystems to be excluded from delegation token renewal at YARN. ### Why are the changes needed? MapReduce jobs can instruct YARN to skip renewal of tokens obtained from certain hosts by specifying the hosts with configuration mapreduce.job.hdfs-servers.token-renewal.exclude=,,..,. But seems Spark lacks of similar option. So the job submission fails if YARN fails to renew DelegationToken for any of the remote HDFS cluster. The failure in DT renewal can happen due to many reason like Remote HDFS does not trust Kerberos identity of YARN etc. We have a customer facing such issue. ### Does this PR introduce _any_ user-facing change? No, if the config is not set. Yes, as users can use this config to instruct YARN not to renew delegation token from certain filesystems. ### How was this patch tested? It is hard to do unit test for this. We did verify it work from the customer using this fix in the production environment. Closes #31761 from viirya/SPARK-34295. Authored-by: Liang-Chi Hsieh Signed-off-by: Liang-Chi Hsieh --- .../HadoopFSDelegationTokenProvider.scala | 22 ++++++++++++++----- .../spark/internal/config/package.scala | 12 ++++++++++ docs/running-on-yarn.md | 12 ++++++++++ docs/security.md | 3 +++ 4 files changed, 44 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala index 0dc6aa1d7ef30..5c98762d4181d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala @@ -48,7 +48,12 @@ private[deploy] class HadoopFSDelegationTokenProvider creds: Credentials): Option[Long] = { try { val fileSystems = HadoopFSDelegationTokenProvider.hadoopFSsToAccess(sparkConf, hadoopConf) - val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fileSystems, creds) + // The hosts on which the file systems to be excluded from token renewal + val fsToExclude = sparkConf.get(YARN_KERBEROS_FILESYSTEM_RENEWAL_EXCLUDE) + .map(new Path(_).getFileSystem(hadoopConf).getUri.getHost) + .toSet + val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fileSystems, creds, + fsToExclude) // Get the token renewal interval if it is not set. It will only be called once. if (tokenRenewalInterval == null) { @@ -99,11 +104,18 @@ private[deploy] class HadoopFSDelegationTokenProvider private def fetchDelegationTokens( renewer: String, filesystems: Set[FileSystem], - creds: Credentials): Credentials = { + creds: Credentials, + fsToExclude: Set[String]): Credentials = { filesystems.foreach { fs => - logInfo(s"getting token for: $fs with renewer $renewer") - fs.addDelegationTokens(renewer, creds) + if (fsToExclude.contains(fs.getUri.getHost)) { + // YARN RM skips renewing token with empty renewer + logInfo(s"getting token for: $fs with empty renewer to skip renewal") + fs.addDelegationTokens("", creds) + } else { + logInfo(s"getting token for: $fs with renewer $renewer") + fs.addDelegationTokens(renewer, creds) + } } creds @@ -119,7 +131,7 @@ private[deploy] class HadoopFSDelegationTokenProvider val renewer = UserGroupInformation.getCurrentUser().getUserName() val creds = new Credentials() - fetchDelegationTokens(renewer, filesystems, creds) + fetchDelegationTokens(renewer, filesystems, creds, Set.empty) val renewIntervals = creds.getAllTokens.asScala.filter { _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier] diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 6b1e3d0c075b0..d988e522c3df7 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -716,6 +716,18 @@ package object config { .toSequence .createWithDefault(Nil) + private[spark] val YARN_KERBEROS_FILESYSTEM_RENEWAL_EXCLUDE = + ConfigBuilder("spark.yarn.kerberos.renewal.excludeHadoopFileSystems") + .doc("The list of Hadoop filesystem URLs whose hosts will be excluded from " + + "delegation token renewal at resource scheduler. Currently this is known to " + + "work under YARN, so YARN Resource Manager won't renew tokens for the application. " + + "Note that as resource scheduler does not renew token, so any application running " + + "longer than the original token expiration that tries to use that token will likely fail.") + .version("3.2.0") + .stringConf + .toSequence + .createWithDefault(Nil) + private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances") .version("1.0.0") .intConf diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 797d18a0d4139..72df64b3efc01 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -699,6 +699,18 @@ staging directory of the Spark application. 2.3.0 + + spark.yarn.kerberos.renewal.excludeHadoopFileSystems + (none) + + A comma-separated list of Hadoop filesystems for whose hosts will be excluded from from delegation + token renewal at resource scheduler. For example, spark.yarn.kerberos.renewal.excludeHadoopFileSystems=hdfs://nn1.com:8032, + hdfs://nn2.com:8032. This is known to work under YARN for now, so YARN Resource Manager won't renew tokens for the application. + Note that as resource scheduler does not renew token, so any application running longer than the original token expiration that tries + to use that token will likely fail. + + 3.2.0 + ## Troubleshooting Kerberos diff --git a/docs/security.md b/docs/security.md index aef6e69382a80..a4ede9f05bafd 100644 --- a/docs/security.md +++ b/docs/security.md @@ -840,6 +840,9 @@ The following options provides finer-grained control for this feature: +Users can exclude Kerberos delegation token renewal at resource scheduler. Currently it is only supported +on YARN. The configuration is covered in the [Running Spark on YARN](running-on-yarn.html#yarn-specific-kerberos-configuration) page. + ## Long-Running Applications Long-running applications may run into issues if their run time exceeds the maximum delegation From 8ed5808f64e83a9f085d456c6ab9188c49992eae Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Wed, 24 Mar 2021 08:50:45 -0500 Subject: [PATCH 16/24] [SPARK-34488][CORE] Support task Metrics Distributions and executor Metrics Distributions in the REST API call for a specified stage MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? For a specific stage, it is useful to show the task metrics in percentile distribution.  This information can help users know whether or not there is a skew/bottleneck among tasks in a given stage.  We list an example in taskMetricsDistributions.json Similarly, it is useful to show the executor metrics in percentile distribution for a specific stage. This information can show whether or not there is a skewed load on some executors.  We list an example in executorMetricsDistributions.json We define `withSummaries` and `quantiles` query parameter in the REST API for a specific stage as: applications///?withSummaries=[true|false]& quantiles=0.05,0.25,0.5,0.75,0.95 1. withSummaries: default is false, define whether to show current stage's taskMetricsDistribution and executorMetricsDistribution 2. quantiles: default is `0.0,0.25,0.5,0.75,1.0` only effect when `withSummaries=true`, it define the quantiles we use when calculating metrics distributions. When withSummaries=true, both task metrics in percentile distribution and executor metrics in percentile distribution are included in the REST API output.  The default value of withSummaries is false, i.e. no metrics percentile distribution will be included in the REST API output.   ### Why are the changes needed? For a specific stage, it is useful to show the task metrics in percentile distribution.  This information can help users know whether or not there is a skew/bottleneck among tasks in a given stage.  We list an example in taskMetricsDistributions.json ### Does this PR introduce _any_ user-facing change? User can use below restful API to get task metrics distribution and executor metrics distribution for indivial stage ``` applications///?withSummaries=[true|false] ``` ### How was this patch tested? Added UT Closes #31611 from AngersZhuuuu/SPARK-34488. Authored-by: Angerszhuuuu Signed-off-by: Sean Owen --- .../apache/spark/status/AppStatusStore.scala | 206 +++- .../org/apache/spark/status/LiveEntity.scala | 4 +- .../spark/status/api/v1/StagesResource.scala | 38 +- .../org/apache/spark/status/api/v1/api.scala | 52 +- .../org/apache/spark/ui/jobs/JobPage.scala | 4 +- .../stage_with_summaries_expectation.json | 1077 +++++++++++++++++ .../deploy/history/HistoryServerSuite.scala | 1 + .../org/apache/spark/ui/StagePageSuite.scala | 4 +- docs/monitoring.md | 18 +- 9 files changed, 1326 insertions(+), 78 deletions(-) create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_with_summaries_expectation.json diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index b9cc9145feb4d..8d43befd745a8 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -113,10 +113,15 @@ private[spark] class AppStatusStore( } } - def stageData(stageId: Int, details: Boolean = false): Seq[v1.StageData] = { + def stageData( + stageId: Int, + details: Boolean = false, + withSummaries: Boolean = false, + unsortedQuantiles: Array[Double] = Array.empty[Double]): Seq[v1.StageData] = { store.view(classOf[StageDataWrapper]).index("stageId").first(stageId).last(stageId) .asScala.map { s => - if (details) stageWithDetails(s.info) else s.info + newStageData(s.info, withDetail = details, withSummaries = withSummaries, + unsortedQuantiles = unsortedQuantiles) }.toSeq } @@ -138,11 +143,15 @@ private[spark] class AppStatusStore( } } - def stageAttempt(stageId: Int, stageAttemptId: Int, - details: Boolean = false): (v1.StageData, Seq[Int]) = { + def stageAttempt( + stageId: Int, stageAttemptId: Int, + details: Boolean = false, + withSummaries: Boolean = false, + unsortedQuantiles: Array[Double] = Array.empty[Double]): (v1.StageData, Seq[Int]) = { val stageKey = Array(stageId, stageAttemptId) val stageDataWrapper = store.read(classOf[StageDataWrapper], stageKey) - val stage = if (details) stageWithDetails(stageDataWrapper.info) else stageDataWrapper.info + val stage = newStageData(stageDataWrapper.info, withDetail = details, + withSummaries = withSummaries, unsortedQuantiles = unsortedQuantiles) (stage, stageDataWrapper.jobIds.toSeq) } @@ -453,61 +462,138 @@ private[spark] class AppStatusStore( } } - private def stageWithDetails(stage: v1.StageData): v1.StageData = { - val tasks = taskList(stage.stageId, stage.attemptId, Int.MaxValue) - .map { t => (t.taskId, t) } - .toMap - - new v1.StageData( - status = stage.status, - stageId = stage.stageId, - attemptId = stage.attemptId, - numTasks = stage.numTasks, - numActiveTasks = stage.numActiveTasks, - numCompleteTasks = stage.numCompleteTasks, - numFailedTasks = stage.numFailedTasks, - numKilledTasks = stage.numKilledTasks, - numCompletedIndices = stage.numCompletedIndices, - submissionTime = stage.submissionTime, - firstTaskLaunchedTime = stage.firstTaskLaunchedTime, - completionTime = stage.completionTime, - failureReason = stage.failureReason, - executorDeserializeTime = stage.executorDeserializeTime, - executorDeserializeCpuTime = stage.executorDeserializeCpuTime, - executorRunTime = stage.executorRunTime, - executorCpuTime = stage.executorCpuTime, - resultSize = stage.resultSize, - jvmGcTime = stage.jvmGcTime, - resultSerializationTime = stage.resultSerializationTime, - memoryBytesSpilled = stage.memoryBytesSpilled, - diskBytesSpilled = stage.diskBytesSpilled, - peakExecutionMemory = stage.peakExecutionMemory, - inputBytes = stage.inputBytes, - inputRecords = stage.inputRecords, - outputBytes = stage.outputBytes, - outputRecords = stage.outputRecords, - shuffleRemoteBlocksFetched = stage.shuffleRemoteBlocksFetched, - shuffleLocalBlocksFetched = stage.shuffleLocalBlocksFetched, - shuffleFetchWaitTime = stage.shuffleFetchWaitTime, - shuffleRemoteBytesRead = stage.shuffleRemoteBytesRead, - shuffleRemoteBytesReadToDisk = stage.shuffleRemoteBytesReadToDisk, - shuffleLocalBytesRead = stage.shuffleLocalBytesRead, - shuffleReadBytes = stage.shuffleReadBytes, - shuffleReadRecords = stage.shuffleReadRecords, - shuffleWriteBytes = stage.shuffleWriteBytes, - shuffleWriteTime = stage.shuffleWriteTime, - shuffleWriteRecords = stage.shuffleWriteRecords, - name = stage.name, - description = stage.description, - details = stage.details, - schedulingPool = stage.schedulingPool, - rddIds = stage.rddIds, - accumulatorUpdates = stage.accumulatorUpdates, - tasks = Some(tasks), - executorSummary = Some(executorSummary(stage.stageId, stage.attemptId)), - killedTasksSummary = stage.killedTasksSummary, - resourceProfileId = stage.resourceProfileId, - peakExecutorMetrics = stage.peakExecutorMetrics) + def newStageData( + stage: v1.StageData, + withDetail: Boolean = false, + withSummaries: Boolean = false, + unsortedQuantiles: Array[Double] = Array.empty[Double]): v1.StageData = { + if (!withDetail && !withSummaries) { + stage + } else { + val quantiles = unsortedQuantiles.sorted + val tasks: Option[Map[Long, v1.TaskData]] = if (withDetail) { + val tasks = taskList(stage.stageId, stage.attemptId, Int.MaxValue) + .map { t => (t.taskId, t) } + .toMap + Some(tasks) + } else { + None + } + val executorSummaries: Option[Map[String, v1.ExecutorStageSummary]] = if (withDetail) { + Some(executorSummary(stage.stageId, stage.attemptId)) + } else { + None + } + val taskMetricsDistribution: Option[v1.TaskMetricDistributions] = if (withSummaries) { + taskSummary(stage.stageId, stage.attemptId, quantiles) + } else { + None + } + val executorMetricsDistributions: Option[v1.ExecutorMetricsDistributions] = + if (withSummaries) { + stageExecutorSummary(stage.stageId, stage.attemptId, quantiles) + } else { + None + } + + new v1.StageData( + status = stage.status, + stageId = stage.stageId, + attemptId = stage.attemptId, + numTasks = stage.numTasks, + numActiveTasks = stage.numActiveTasks, + numCompleteTasks = stage.numCompleteTasks, + numFailedTasks = stage.numFailedTasks, + numKilledTasks = stage.numKilledTasks, + numCompletedIndices = stage.numCompletedIndices, + submissionTime = stage.submissionTime, + firstTaskLaunchedTime = stage.firstTaskLaunchedTime, + completionTime = stage.completionTime, + failureReason = stage.failureReason, + executorDeserializeTime = stage.executorDeserializeTime, + executorDeserializeCpuTime = stage.executorDeserializeCpuTime, + executorRunTime = stage.executorRunTime, + executorCpuTime = stage.executorCpuTime, + resultSize = stage.resultSize, + jvmGcTime = stage.jvmGcTime, + resultSerializationTime = stage.resultSerializationTime, + memoryBytesSpilled = stage.memoryBytesSpilled, + diskBytesSpilled = stage.diskBytesSpilled, + peakExecutionMemory = stage.peakExecutionMemory, + inputBytes = stage.inputBytes, + inputRecords = stage.inputRecords, + outputBytes = stage.outputBytes, + outputRecords = stage.outputRecords, + shuffleRemoteBlocksFetched = stage.shuffleRemoteBlocksFetched, + shuffleLocalBlocksFetched = stage.shuffleLocalBlocksFetched, + shuffleFetchWaitTime = stage.shuffleFetchWaitTime, + shuffleRemoteBytesRead = stage.shuffleRemoteBytesRead, + shuffleRemoteBytesReadToDisk = stage.shuffleRemoteBytesReadToDisk, + shuffleLocalBytesRead = stage.shuffleLocalBytesRead, + shuffleReadBytes = stage.shuffleReadBytes, + shuffleReadRecords = stage.shuffleReadRecords, + shuffleWriteBytes = stage.shuffleWriteBytes, + shuffleWriteTime = stage.shuffleWriteTime, + shuffleWriteRecords = stage.shuffleWriteRecords, + name = stage.name, + description = stage.description, + details = stage.details, + schedulingPool = stage.schedulingPool, + rddIds = stage.rddIds, + accumulatorUpdates = stage.accumulatorUpdates, + tasks = tasks, + executorSummary = executorSummaries, + killedTasksSummary = stage.killedTasksSummary, + resourceProfileId = stage.resourceProfileId, + peakExecutorMetrics = stage.peakExecutorMetrics, + taskMetricsDistributions = taskMetricsDistribution, + executorMetricsDistributions = executorMetricsDistributions) + } + } + + def stageExecutorSummary( + stageId: Int, + stageAttemptId: Int, + unsortedQuantiles: Array[Double]): Option[v1.ExecutorMetricsDistributions] = { + val quantiles = unsortedQuantiles.sorted + val summary = executorSummary(stageId, stageAttemptId) + if (summary.isEmpty) { + None + } else { + val values = summary.values.toIndexedSeq + Some(new v1.ExecutorMetricsDistributions( + quantiles = quantiles, + taskTime = getQuantilesValue(values.map(_.taskTime.toDouble).sorted, quantiles), + failedTasks = getQuantilesValue(values.map(_.failedTasks.toDouble).sorted, quantiles), + succeededTasks = getQuantilesValue(values.map(_.succeededTasks.toDouble).sorted, quantiles), + killedTasks = getQuantilesValue(values.map(_.killedTasks.toDouble).sorted, quantiles), + inputBytes = getQuantilesValue(values.map(_.inputBytes.toDouble).sorted, quantiles), + inputRecords = getQuantilesValue(values.map(_.inputRecords.toDouble).sorted, quantiles), + outputBytes = getQuantilesValue(values.map(_.outputBytes.toDouble).sorted, quantiles), + outputRecords = getQuantilesValue(values.map(_.outputRecords.toDouble).sorted, quantiles), + shuffleRead = getQuantilesValue(values.map(_.shuffleRead.toDouble).sorted, quantiles), + shuffleReadRecords = + getQuantilesValue(values.map(_.shuffleReadRecords.toDouble).sorted, quantiles), + shuffleWrite = getQuantilesValue(values.map(_.shuffleWrite.toDouble).sorted, quantiles), + shuffleWriteRecords = + getQuantilesValue(values.map(_.shuffleWriteRecords.toDouble).sorted, quantiles), + memoryBytesSpilled = + getQuantilesValue(values.map(_.memoryBytesSpilled.toDouble).sorted, quantiles), + diskBytesSpilled = + getQuantilesValue(values.map(_.diskBytesSpilled.toDouble).sorted, quantiles), + peakMemoryMetrics = + new v1.ExecutorPeakMetricsDistributions(quantiles, + values.flatMap(_.peakMemoryMetrics)) + )) + } + } + + def getQuantilesValue( + values: IndexedSeq[Double], + quantiles: Array[Double]): IndexedSeq[Double] = { + val count = values.size + val indices = quantiles.map { q => math.min((q * count).toLong, count - 1) } + indices.map(i => values(i.toInt)).toIndexedSeq } def rdd(rddId: Int): v1.RDDStorageInfo = { diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 3105e7b5e77da..b6dfe30bb889e 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -493,7 +493,9 @@ private class LiveStage extends LiveEntity { executorSummary = None, killedTasksSummary = killedSummary, resourceProfileId = info.resourceProfileId, - Some(peakExecutorMetrics).filter(_.isSet)) + peakExecutorMetrics = Some(peakExecutorMetrics).filter(_.isSet), + taskMetricsDistributions = None, + executorMetricsDistributions = None) } override protected def doUpdate(): Any = { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 84bd430d9abef..be7a4a681898e 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -36,9 +36,14 @@ private[v1] class StagesResource extends BaseAppResource { @Path("{stageId: \\d+}") def stageData( @PathParam("stageId") stageId: Int, - @QueryParam("details") @DefaultValue("true") details: Boolean): Seq[StageData] = { + @QueryParam("details") @DefaultValue("true") details: Boolean, + @QueryParam("withSummaries") @DefaultValue("false") withSummaries: Boolean, + @QueryParam("quantiles") @DefaultValue("0.0,0.25,0.5,0.75,1.0") quantileString: String): + Seq[StageData] = { withUI { ui => - val ret = ui.store.stageData(stageId, details = details) + val quantiles = parseQuantileString(quantileString) + val ret = ui.store.stageData(stageId, details = details, + withSummaries = withSummaries, unsortedQuantiles = quantiles) if (ret.nonEmpty) { ret } else { @@ -52,9 +57,14 @@ private[v1] class StagesResource extends BaseAppResource { def oneAttemptData( @PathParam("stageId") stageId: Int, @PathParam("stageAttemptId") stageAttemptId: Int, - @QueryParam("details") @DefaultValue("true") details: Boolean): StageData = withUI { ui => + @QueryParam("details") @DefaultValue("true") details: Boolean, + @QueryParam("withSummaries") @DefaultValue("false") withSummaries: Boolean, + @QueryParam("quantiles") @DefaultValue("0.0,0.25,0.5,0.75,1.0") quantileString: String): + StageData = withUI { ui => try { - ui.store.stageAttempt(stageId, stageAttemptId, details = details)._1 + val quantiles = parseQuantileString(quantileString) + ui.store.stageAttempt(stageId, stageAttemptId, details = details, + withSummaries = withSummaries, unsortedQuantiles = quantiles)._1 } catch { case _: NoSuchElementException => // Change the message depending on whether there are any attempts for the requested stage. @@ -76,15 +86,7 @@ private[v1] class StagesResource extends BaseAppResource { @PathParam("stageAttemptId") stageAttemptId: Int, @DefaultValue("0.05,0.25,0.5,0.75,0.95") @QueryParam("quantiles") quantileString: String) : TaskMetricDistributions = withUI { ui => - val quantiles = quantileString.split(",").map { s => - try { - s.toDouble - } catch { - case nfe: NumberFormatException => - throw new BadParameterException("quantiles", "double", s) - } - } - + val quantiles = parseQuantileString(quantileString) ui.store.taskSummary(stageId, stageAttemptId, quantiles).getOrElse( throw new NotFoundException(s"No tasks reported metrics for $stageId / $stageAttemptId yet.")) } @@ -226,4 +228,14 @@ private[v1] class StagesResource extends BaseAppResource { filteredTaskDataSequence } + def parseQuantileString(quantileString: String): Array[Double] = { + quantileString.split(",").map { s => + try { + s.toDouble + } catch { + case nfe: NumberFormatException => + throw new BadParameterException("quantiles", "double", s) + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 96f5b7b5cf27e..52fcb29ebfc14 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -170,6 +170,19 @@ private[spark] class ExecutorMetricsJsonSerializer value.isEmpty } +private[spark] class ExecutorPeakMetricsDistributionsJsonSerializer + extends JsonSerializer[ExecutorPeakMetricsDistributions] { + override def serialize( + metrics: ExecutorPeakMetricsDistributions, + jsonGenerator: JsonGenerator, + serializerProvider: SerializerProvider): Unit = { + val metricsMap = ExecutorMetricType.metricToOffset.map { case (metric, _) => + metric -> metrics.getMetricDistribution(metric) + } + jsonGenerator.writeObject(metricsMap) + } +} + class JobData private[spark]( val jobId: Int, val name: String, @@ -279,7 +292,9 @@ class StageData private[spark]( val resourceProfileId: Int, @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) - val peakExecutorMetrics: Option[ExecutorMetrics]) + val peakExecutorMetrics: Option[ExecutorMetrics], + val taskMetricsDistributions: Option[TaskMetricDistributions], + val executorMetricsDistributions: Option[ExecutorMetricsDistributions]) class TaskData private[spark]( val taskId: Long, @@ -368,6 +383,41 @@ class OutputMetricDistributions private[spark]( val bytesWritten: IndexedSeq[Double], val recordsWritten: IndexedSeq[Double]) +class ExecutorMetricsDistributions private[spark]( + val quantiles: IndexedSeq[Double], + + val taskTime: IndexedSeq[Double], + val failedTasks: IndexedSeq[Double], + val succeededTasks: IndexedSeq[Double], + val killedTasks: IndexedSeq[Double], + val inputBytes: IndexedSeq[Double], + val inputRecords: IndexedSeq[Double], + val outputBytes: IndexedSeq[Double], + val outputRecords: IndexedSeq[Double], + val shuffleRead: IndexedSeq[Double], + val shuffleReadRecords: IndexedSeq[Double], + val shuffleWrite: IndexedSeq[Double], + val shuffleWriteRecords: IndexedSeq[Double], + val memoryBytesSpilled: IndexedSeq[Double], + val diskBytesSpilled: IndexedSeq[Double], + @JsonSerialize(using = classOf[ExecutorPeakMetricsDistributionsJsonSerializer]) + val peakMemoryMetrics: ExecutorPeakMetricsDistributions +) + +@JsonSerialize(using = classOf[ExecutorPeakMetricsDistributionsJsonSerializer]) +class ExecutorPeakMetricsDistributions private[spark]( + val quantiles: IndexedSeq[Double], + val executorMetrics: IndexedSeq[ExecutorMetrics]) { + private lazy val count = executorMetrics.length + private lazy val indices = quantiles.map { q => math.min((q * count).toLong, count - 1) } + + /** Returns the distributions for the specified metric. */ + def getMetricDistribution(metricName: String): IndexedSeq[Double] = { + val sorted = executorMetrics.map(_.getMetricValue(metricName)).sorted + indices.map(i => sorted(i.toInt).toDouble).toIndexedSeq + } +} + class ShuffleReadMetricDistributions private[spark]( val readBytes: IndexedSeq[Double], val readRecords: IndexedSeq[Double], diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 1dfbce82c852b..a19daec0f1453 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -257,7 +257,9 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP executorSummary = None, killedTasksSummary = Map(), ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID, - peakExecutorMetrics = None) + peakExecutorMetrics = None, + taskMetricsDistributions = None, + executorMetricsDistributions = None) } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_summaries_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_summaries_expectation.json new file mode 100644 index 0000000000000..179e46faa6b3d --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_summaries_expectation.json @@ -0,0 +1,1077 @@ +{ + "status" : "COMPLETE", + "stageId" : 2, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:21.040GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:21.077GMT", + "completionTime" : "2020-07-07T03:11:23.044GMT", + "executorDeserializeTime" : 3905, + "executorDeserializeCpuTime" : 979900000, + "executorRunTime" : 25579, + "executorCpuTime" : 8810338000, + "resultSize" : 33883, + "jvmGcTime" : 1010, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 384640, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "foreach at :26", + "details" : "org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)", + "schedulingPool" : "default", + "rddIds" : [ 10, 8, 6, 7, 9 ], + "accumulatorUpdates" : [ ], + "tasks" : { + "42" : { + "taskId" : 42, + "index" : 10, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.120GMT", + "duration" : 1923, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 229, + "executorDeserializeCpuTime" : 58152000, + "executorRunTime" : 1624, + "executorCpuTime" : 508230000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 70, + "gettingResultTime" : 0 + }, + "37" : { + "taskId" : 37, + "index" : 5, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.100GMT", + "duration" : 1915, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 256, + "executorDeserializeCpuTime" : 60890000, + "executorRunTime" : 1596, + "executorCpuTime" : 507192000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 63, + "gettingResultTime" : 0 + }, + "46" : { + "taskId" : 46, + "index" : 14, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.132GMT", + "duration" : 1905, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 218, + "executorDeserializeCpuTime" : 51464000, + "executorRunTime" : 1618, + "executorCpuTime" : 490927000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 69, + "gettingResultTime" : 0 + }, + "38" : { + "taskId" : 38, + "index" : 6, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.104GMT", + "duration" : 1835, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 255, + "executorDeserializeCpuTime" : 60358000, + "executorRunTime" : 1498, + "executorCpuTime" : 414110000, + "resultSize" : 2158, + "jvmGcTime" : 62, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + }, + "33" : { + "taskId" : 33, + "index" : 1, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.082GMT", + "duration" : 1943, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 267, + "executorDeserializeCpuTime" : 54442000, + "executorRunTime" : 1597, + "executorCpuTime" : 519178000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 79, + "gettingResultTime" : 0 + }, + "41" : { + "taskId" : 41, + "index" : 9, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.116GMT", + "duration" : 1916, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 240, + "executorDeserializeCpuTime" : 55787000, + "executorRunTime" : 1614, + "executorCpuTime" : 489923000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 62, + "gettingResultTime" : 0 + }, + "32" : { + "taskId" : 32, + "index" : 0, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.077GMT", + "duration" : 1960, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 271, + "executorDeserializeCpuTime" : 56827000, + "executorRunTime" : 1619, + "executorCpuTime" : 496683000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 70, + "gettingResultTime" : 0 + }, + "34" : { + "taskId" : 34, + "index" : 2, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.087GMT", + "duration" : 1939, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 265, + "executorDeserializeCpuTime" : 69492000, + "executorRunTime" : 1606, + "executorCpuTime" : 508433000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 68, + "gettingResultTime" : 0 + }, + "45" : { + "taskId" : 45, + "index" : 13, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.129GMT", + "duration" : 1895, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 221, + "executorDeserializeCpuTime" : 54222000, + "executorRunTime" : 1595, + "executorCpuTime" : 495138000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 79, + "gettingResultTime" : 0 + }, + "44" : { + "taskId" : 44, + "index" : 12, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.126GMT", + "duration" : 1917, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 222, + "executorDeserializeCpuTime" : 51988000, + "executorRunTime" : 1624, + "executorCpuTime" : 498187000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + }, + "39" : { + "taskId" : 39, + "index" : 7, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.109GMT", + "duration" : 1915, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 254, + "executorDeserializeCpuTime" : 64380000, + "executorRunTime" : 1596, + "executorCpuTime" : 539451000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 65, + "gettingResultTime" : 0 + }, + "35" : { + "taskId" : 35, + "index" : 3, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.091GMT", + "duration" : 1925, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 263, + "executorDeserializeCpuTime" : 62944000, + "executorRunTime" : 1598, + "executorCpuTime" : 502908000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 64, + "gettingResultTime" : 0 + }, + "43" : { + "taskId" : 43, + "index" : 11, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.123GMT", + "duration" : 1906, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 225, + "executorDeserializeCpuTime" : 48849000, + "executorRunTime" : 1609, + "executorCpuTime" : 502120000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 72, + "gettingResultTime" : 0 + }, + "40" : { + "taskId" : 40, + "index" : 8, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.112GMT", + "duration" : 1904, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 246, + "executorDeserializeCpuTime" : 69760000, + "executorRunTime" : 1595, + "executorCpuTime" : 510597000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 63, + "gettingResultTime" : 0 + }, + "36" : { + "taskId" : 36, + "index" : 4, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.095GMT", + "duration" : 1920, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 260, + "executorDeserializeCpuTime" : 112849000, + "executorRunTime" : 1596, + "executorCpuTime" : 503010000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 64, + "gettingResultTime" : 0 + }, + "47" : { + "taskId" : 47, + "index" : 15, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.136GMT", + "duration" : 1878, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 213, + "executorDeserializeCpuTime" : 47496000, + "executorRunTime" : 1594, + "executorCpuTime" : 1324251000, + "resultSize" : 2115, + "jvmGcTime" : 52, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + } + }, + "executorSummary" : { + "0" : { + "taskTime" : 30596, + "failedTasks" : 0, + "succeededTasks" : 16, + "killedTasks" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRead" : 0, + "shuffleReadRecords" : 0, + "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false, + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, + "isExcludedForStage" : false + }, + "driver" : { + "taskTime" : 0, + "failedTasks" : 0, + "succeededTasks" : 0, + "killedTasks" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRead" : 0, + "shuffleReadRecords" : 0, + "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false, + "peakMemoryMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + }, + "isExcludedForStage" : false + } + }, + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + }, + "taskMetricsDistributions" : { + "quantiles" : [ 0.0, 0.25, 0.5, 0.75, 1.0 ], + "executorDeserializeTime" : [ 213.0, 225.0, 254.0, 263.0, 271.0 ], + "executorDeserializeCpuTime" : [ 4.7496E7, 5.4222E7, 5.8152E7, 6.438E7, 1.12849E8 ], + "executorRunTime" : [ 1498.0, 1596.0, 1598.0, 1618.0, 1624.0 ], + "executorCpuTime" : [ 4.1411E8, 4.96683E8, 5.0301E8, 5.10597E8, 1.324251E9 ], + "resultSize" : [ 2115.0, 2115.0, 2115.0, 2115.0, 2158.0 ], + "jvmGcTime" : [ 52.0, 62.0, 62.0, 66.0, 66.0 ], + "resultSerializationTime" : [ 0.0, 0.0, 0.0, 0.0, 11.0 ], + "gettingResultTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "schedulerDelay" : [ 62.0, 64.0, 70.0, 71.0, 79.0 ], + "peakExecutionMemory" : [ 24040.0, 24040.0, 24040.0, 24040.0, 24040.0 ], + "memoryBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "diskBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "inputMetrics" : { + "bytesRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "recordsRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] + }, + "outputMetrics" : { + "bytesWritten" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "recordsWritten" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] + }, + "shuffleReadMetrics" : { + "readBytes" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "readRecords" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "remoteBlocksFetched" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "localBlocksFetched" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "fetchWaitTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "remoteBytesRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "remoteBytesReadToDisk" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "totalBlocksFetched" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] + }, + "shuffleWriteMetrics" : { + "writeBytes" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "writeRecords" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "writeTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] + } + }, + "executorMetricsDistributions" : { + "quantiles" : [ 0.0, 0.25, 0.5, 0.75, 1.0 ], + "taskTime" : [ 0.0, 0.0, 30596.0, 30596.0, 30596.0 ], + "failedTasks" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "succeededTasks" : [ 0.0, 0.0, 16.0, 16.0, 16.0 ], + "killedTasks" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "inputBytes" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "inputRecords" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "outputBytes" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "outputRecords" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "shuffleRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "shuffleReadRecords" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "shuffleWrite" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "shuffleWriteRecords" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "memoryBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "diskBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : [ 0.0, 0.0, 2.13367864E8, 2.13367864E8, 2.13367864E8 ], + "JVMOffHeapMemory" : [ 0.0, 0.0, 1.89011656E8, 1.89011656E8, 1.89011656E8 ], + "OnHeapExecutionMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "OffHeapExecutionMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "OnHeapStorageMemory" : [ 0.0, 0.0, 2133349.0, 2133349.0, 2133349.0 ], + "OffHeapStorageMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "OnHeapUnifiedMemory" : [ 0.0, 0.0, 2133349.0, 2133349.0, 2133349.0 ], + "OffHeapUnifiedMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "DirectPoolMemory" : [ 0.0, 0.0, 282024.0, 282024.0, 282024.0 ], + "MappedPoolMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "ProcessTreeJVMVMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "ProcessTreeJVMRSSMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "ProcessTreePythonVMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "ProcessTreePythonRSSMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "ProcessTreeOtherVMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "ProcessTreeOtherRSSMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "MinorGCCount" : [ 0.0, 0.0, 13.0, 13.0, 13.0 ], + "MinorGCTime" : [ 0.0, 0.0, 115.0, 115.0, 115.0 ], + "MajorGCCount" : [ 0.0, 0.0, 4.0, 4.0, 4.0 ], + "MajorGCTime" : [ 0.0, 0.0, 339.0, 339.0, 339.0 ] + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 938eb8d13015b..24640b31c9f7a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -181,6 +181,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "multiple resource profiles" -> "applications/application_1578436911597_0052/environment", "stage list with peak metrics" -> "applications/app-20200706201101-0003/stages", "stage with peak metrics" -> "applications/app-20200706201101-0003/stages/2/0", + "stage with summaries" -> "applications/app-20200706201101-0003/stages/2/0?withSummaries=true", "app environment" -> "applications/app-20161116163331-0000/environment", diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index d02d7f862df80..9f0b73f8132f8 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -93,7 +93,9 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { executorSummary = None, killedTasksSummary = Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, - peakExecutorMetrics = None + peakExecutorMetrics = None, + taskMetricsDistributions = None, + executorMetricsDistributions = None ) val taskTable = new TaskPagedTable( stageData, diff --git a/docs/monitoring.md b/docs/monitoring.md index 930f91f9a5c2f..8fec5adc3b12f 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -479,11 +479,27 @@ can be identified by their `[attempt-id]`. In the API listed below, when running /applications/[app-id]/stages/[stage-id] A list of all attempts for the given stage. +
?details=true lists all attempts with the task data for the given stage. +
?withSummaries=true lists task metrics distribution and executor metrics distribution of each attempt. +
?quantiles=0.1,0.25,0.5,0.75,1.0 summarize the metrics with the given quantiles. Query parameter quantiles takes effect only when withSummaries=true. Default value is 0.0,0.25,0.5,0.75,1.0. +
Example: +
?details=true +
?withSummaries=true +
?details=true&withSummaries=true&quantiles=0.01,0.5,0.99 /applications/[app-id]/stages/[stage-id]/[stage-attempt-id] - Details for the given stage attempt. + + Details for the given stage attempt. +
?details=true lists all task data for the given stage attempt. +
?withSummaries=true lists task metrics distribution and executor metrics distribution for the given stage attempt. +
?quantiles=0.1,0.25,0.5,0.75,1.0 summarize the metrics with the given quantiles. Query parameter quantiles takes effect only when withSummaries=true. Default value is 0.0,0.25,0.5,0.75,1.0. +
Example: +
?details=true +
?withSummaries=true +
?details=true&withSummaries=true&quantiles=0.01,0.5,0.99 + /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskSummary From 35c70e417d8c6e3958e0da8a4bec731f9e394a28 Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Wed, 24 Mar 2021 23:06:35 +0900 Subject: [PATCH 17/24] [SPARK-34853][SQL] Remove duplicated definition of output partitioning/ordering for limit operator ### What changes were proposed in this pull request? Both local limit and global limit define the output partitioning and output ordering in the same way and this is duplicated (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala#L159-L175 ). We can move the output partitioning and ordering into their parent trait - `BaseLimitExec`. This is doable as `BaseLimitExec` has no more other child class. This is a minor code refactoring. ### Why are the changes needed? Clean up the code a little bit. Better readability. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pure refactoring. Rely on existing unit tests. Closes #31950 from c21/limit-cleanup. Authored-by: Cheng Su Signed-off-by: Takeshi Yamamuro --- .../org/apache/spark/sql/execution/limit.scala | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index d8f67fb7357e7..e5a299523c79c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -113,6 +113,10 @@ object BaseLimitExec { trait BaseLimitExec extends LimitExec with CodegenSupport { override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter => iter.take(limit) } @@ -156,12 +160,7 @@ trait BaseLimitExec extends LimitExec with CodegenSupport { /** * Take the first `limit` elements of each child partition, but do not collect or shuffle them. */ -case class LocalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { - - override def outputOrdering: Seq[SortOrder] = child.outputOrdering - - override def outputPartitioning: Partitioning = child.outputPartitioning -} +case class LocalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec /** * Take the first `limit` elements of the child's single output partition. @@ -169,10 +168,6 @@ case class LocalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil - - override def outputPartitioning: Partitioning = child.outputPartitioning - - override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** From ad211ccd9da479a7d6d6324b9ea6b52c066788bd Mon Sep 17 00:00:00 2001 From: Danny Meijer Date: Wed, 24 Mar 2021 15:21:19 +0100 Subject: [PATCH 18/24] [SPARK-34630][PYTHON][SQL] Added typehint for pyspark.sql.Column.contains ### What changes were proposed in this pull request? This PR implements the missing typehints as per SPARK-34630. ### Why are the changes needed? To satisfy the aforementioned Jira ticket ### Does this PR introduce _any_ user-facing change? No, just adding a missing typehint for Project Zen ### How was this patch tested? No tests needed (just adding a typehint) Closes #31823 from dannymeijer/feature/SPARK-34630. Authored-by: Danny Meijer Signed-off-by: zero323 --- python/pyspark/sql/column.pyi | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/sql/column.pyi b/python/pyspark/sql/column.pyi index 1f63e65b3de81..36c1bcc1961a8 100644 --- a/python/pyspark/sql/column.pyi +++ b/python/pyspark/sql/column.pyi @@ -115,3 +115,4 @@ class Column: def over(self, window: WindowSpec) -> Column: ... def __nonzero__(self) -> None: ... def __bool__(self) -> None: ... + def contains(self, item: Any) -> Column: ... From 84df54b4951e4a78ea971c14124a8793dd5ac62d Mon Sep 17 00:00:00 2001 From: Tanel Kiis Date: Wed, 24 Mar 2021 14:36:51 +0000 Subject: [PATCH 19/24] [SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes ### What changes were proposed in this pull request? Update the plan stability golden files even if only the `explain.txt` changes. ### Why are the changes needed? Currently only `simplified.txt` change is checked. There are some PRs, that update the `explain.txt`, that do not change the `simplified.txt`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The updated golden files. Closes #31927 from tanelk/SPARK-34822_update_plan_stability. Lead-authored-by: Tanel Kiis Co-authored-by: tanel.kiis@gmail.com Signed-off-by: Wenchen Fan --- .../q34.sf100/explain.txt | 4 ++-- .../approved-plans-modified/q34/explain.txt | 4 ++-- .../q53.sf100/explain.txt | 6 ++--- .../approved-plans-modified/q53/explain.txt | 6 ++--- .../q63.sf100/explain.txt | 6 ++--- .../approved-plans-modified/q63/explain.txt | 6 ++--- .../q7.sf100/explain.txt | 4 ++-- .../approved-plans-modified/q7/explain.txt | 4 ++-- .../q73.sf100/explain.txt | 4 ++-- .../approved-plans-modified/q73/explain.txt | 4 ++-- .../q89.sf100/explain.txt | 6 ++--- .../approved-plans-modified/q89/explain.txt | 6 ++--- .../q98.sf100/explain.txt | 6 ++--- .../approved-plans-modified/q98/explain.txt | 6 ++--- .../approved-plans-v1_4/q12.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q12/explain.txt | 6 ++--- .../approved-plans-v1_4/q13.sf100/explain.txt | 8 +++---- .../approved-plans-v1_4/q13/explain.txt | 8 +++---- .../approved-plans-v1_4/q16.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q16/explain.txt | 2 +- .../approved-plans-v1_4/q17.sf100/explain.txt | 8 +++---- .../approved-plans-v1_4/q17/explain.txt | 8 +++---- .../approved-plans-v1_4/q18.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q18/explain.txt | 4 ++-- .../approved-plans-v1_4/q20.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q20/explain.txt | 6 ++--- .../approved-plans-v1_4/q21.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q21/explain.txt | 2 +- .../q24a.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q24a/explain.txt | 4 ++-- .../q24b.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q24b/explain.txt | 4 ++-- .../approved-plans-v1_4/q26.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q26/explain.txt | 4 ++-- .../approved-plans-v1_4/q27.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q27/explain.txt | 4 ++-- .../approved-plans-v1_4/q32.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q32/explain.txt | 2 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q33/explain.txt | 4 ++-- .../approved-plans-v1_4/q34.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q34/explain.txt | 4 ++-- .../approved-plans-v1_4/q37.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q37/explain.txt | 2 +- .../approved-plans-v1_4/q38.sf100/explain.txt | 24 +++++++++---------- .../approved-plans-v1_4/q38/explain.txt | 12 +++++----- .../q39a.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q39a/explain.txt | 4 ++-- .../q39b.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q39b/explain.txt | 4 ++-- .../approved-plans-v1_4/q41.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q41/explain.txt | 4 ++-- .../approved-plans-v1_4/q44.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q44/explain.txt | 4 ++-- .../approved-plans-v1_4/q48.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q48/explain.txt | 6 ++--- .../approved-plans-v1_4/q5.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q5/explain.txt | 2 +- .../approved-plans-v1_4/q53.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q53/explain.txt | 6 ++--- .../approved-plans-v1_4/q54.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q54/explain.txt | 4 ++-- .../approved-plans-v1_4/q58.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q58/explain.txt | 2 +- .../approved-plans-v1_4/q63.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q63/explain.txt | 6 ++--- .../approved-plans-v1_4/q64.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q64/explain.txt | 4 ++-- .../approved-plans-v1_4/q67.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q67/explain.txt | 2 +- .../approved-plans-v1_4/q7.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q7/explain.txt | 4 ++-- .../approved-plans-v1_4/q70.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q70/explain.txt | 2 +- .../approved-plans-v1_4/q71.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q71/explain.txt | 4 ++-- .../approved-plans-v1_4/q72.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q72/explain.txt | 4 ++-- .../approved-plans-v1_4/q73.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q73/explain.txt | 4 ++-- .../approved-plans-v1_4/q75.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q75/explain.txt | 4 ++-- .../approved-plans-v1_4/q80.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q80/explain.txt | 2 +- .../approved-plans-v1_4/q82.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q82/explain.txt | 2 +- .../approved-plans-v1_4/q85.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q85/explain.txt | 6 ++--- .../approved-plans-v1_4/q87.sf100/explain.txt | 24 +++++++++---------- .../approved-plans-v1_4/q87/explain.txt | 12 +++++----- .../approved-plans-v1_4/q89.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q89/explain.txt | 6 ++--- .../approved-plans-v1_4/q91.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q91/explain.txt | 4 ++-- .../approved-plans-v1_4/q92.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q92/explain.txt | 2 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q93/explain.txt | 4 ++-- .../approved-plans-v1_4/q94.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q94/explain.txt | 6 ++--- .../approved-plans-v1_4/q95.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q95/explain.txt | 6 ++--- .../approved-plans-v1_4/q98.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q98/explain.txt | 6 ++--- .../approved-plans-v2_7/q12.sf100/explain.txt | 6 ++--- .../approved-plans-v2_7/q12/explain.txt | 6 ++--- .../approved-plans-v2_7/q20.sf100/explain.txt | 6 ++--- .../approved-plans-v2_7/q20/explain.txt | 6 ++--- .../approved-plans-v2_7/q24.sf100/explain.txt | 4 ++-- .../approved-plans-v2_7/q24/explain.txt | 4 ++-- .../approved-plans-v2_7/q34.sf100/explain.txt | 4 ++-- .../approved-plans-v2_7/q34/explain.txt | 4 ++-- .../approved-plans-v2_7/q5a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q5a/explain.txt | 2 +- .../approved-plans-v2_7/q64.sf100/explain.txt | 4 ++-- .../approved-plans-v2_7/q64/explain.txt | 4 ++-- .../q67a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q67a/explain.txt | 2 +- .../q70a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q70a/explain.txt | 2 +- .../approved-plans-v2_7/q72.sf100/explain.txt | 4 ++-- .../approved-plans-v2_7/q72/explain.txt | 4 ++-- .../approved-plans-v2_7/q75.sf100/explain.txt | 4 ++-- .../approved-plans-v2_7/q75/explain.txt | 4 ++-- .../q80a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q80a/explain.txt | 2 +- .../approved-plans-v2_7/q98.sf100/explain.txt | 6 ++--- .../approved-plans-v2_7/q98/explain.txt | 6 ++--- .../apache/spark/sql/PlanStabilitySuite.scala | 18 +++++++------- 129 files changed, 306 insertions(+), 306 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt index ff33c0b00120b..ebc6009080bfa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt index 5ebb8e180e1e0..cd556c0bccb46 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt index 92db706d594f2..a3223d60c96be 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt index 8b29f7c168765..172e66022046d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt index 8e0aaf2c479b8..75cdaecea7595 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt index 712ba76f2878a..c8e0821dfacda 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt index fe4db3541cbd9..daa0ca7b16562 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt @@ -116,7 +116,7 @@ Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_p Output [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary ), IsNotNull(cd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -124,7 +124,7 @@ Input [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_stat (20) Filter [codegen id : 3] Input [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20] -Condition : ((((((isnotnull(cd_gender#18) AND isnotnull(cd_marital_status#19)) AND isnotnull(cd_education_status#20)) AND (cd_gender#18 = F)) AND (cd_marital_status#19 = W)) AND (cd_education_status#20 = Primary)) AND isnotnull(cd_demo_sk#17)) +Condition : ((((((isnotnull(cd_gender#18) AND isnotnull(cd_marital_status#19)) AND isnotnull(cd_education_status#20)) AND (cd_gender#18 = F)) AND (cd_marital_status#19 = W)) AND (cd_education_status#20 = Primary )) AND isnotnull(cd_demo_sk#17)) (21) Project [codegen id : 3] Output [1]: [cd_demo_sk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt index c1f0c1bd01451..a9da9118323b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt index 9dc7b54bf5bb6..c2909d9f5531f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt @@ -60,7 +60,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint) Output [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (6) ColumnarToRow @@ -68,7 +68,7 @@ Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#1 (7) Filter Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] -Condition : (((((isnotnull(hd_vehicle_count#10) AND isnotnull(hd_dep_count#9)) AND ((hd_buy_potential#8 = >10000) OR (hd_buy_potential#8 = Unknown))) AND (hd_vehicle_count#10 > 0)) AND ((cast(hd_dep_count#9 as double) / cast(hd_vehicle_count#10 as double)) > 1.0)) AND isnotnull(hd_demo_sk#7)) +Condition : (((((isnotnull(hd_vehicle_count#10) AND isnotnull(hd_dep_count#9)) AND ((hd_buy_potential#8 = >10000 ) OR (hd_buy_potential#8 = Unknown ))) AND (hd_vehicle_count#10 > 0)) AND ((cast(hd_dep_count#9 as double) / cast(hd_vehicle_count#10 as double)) > 1.0)) AND isnotnull(hd_demo_sk#7)) (8) Project Output [1]: [hd_demo_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt index 559ef93fb70ac..5ec772ca3d638 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt index 393b6a4da4255..a325bf820a4f9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt @@ -109,7 +109,7 @@ Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, d_moy#8, s_store_sk#1 Output [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Home,Books,Electronics]),In(i_class, [wallpaper,parenting,musical])),And(In(i_category, [Shoes,Jewelry,Men]),In(i_class, [womens,birdal,pants]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Home ,Books ,Electronics ]),In(i_class, [wallpaper ,parenting ,musical ])),And(In(i_category, [Shoes ,Jewelry ,Men ]),In(i_class, [womens ,birdal ,pants ]))), IsNotNull(i_item_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -117,7 +117,7 @@ Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] (19) Filter [codegen id : 3] Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] -Condition : (((i_category#17 IN (Home,Books,Electronics) AND i_class#16 IN (wallpaper,parenting,musical)) OR (i_category#17 IN (Shoes,Jewelry,Men) AND i_class#16 IN (womens,birdal,pants))) AND isnotnull(i_item_sk#14)) +Condition : (((i_category#17 IN (Home ,Books ,Electronics ) AND i_class#16 IN (wallpaper ,parenting ,musical )) OR (i_category#17 IN (Shoes ,Jewelry ,Men ) AND i_class#16 IN (womens ,birdal ,pants ))) AND isnotnull(i_item_sk#14)) (20) BroadcastExchange Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#17, i_brand#15, s_store_ (29) Filter [codegen id : 7] Input [9]: [i_category#17, i_class#16, i_brand#15, s_store_name#11, s_company_name#12, d_moy#8, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (30) Project [codegen id : 7] Output [8]: [i_category#17, i_class#16, i_brand#15, s_store_name#11, s_company_name#12, d_moy#8, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt index 0210877799a98..2ec1ea034361a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Home,Books,Electronics]),In(i_class, [wallpaper,parenting,musical])),And(In(i_category, [Shoes,Jewelry,Men]),In(i_class, [womens,birdal,pants]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Home ,Books ,Electronics ]),In(i_class, [wallpaper ,parenting ,musical ])),And(In(i_category, [Shoes ,Jewelry ,Men ]),In(i_class, [womens ,birdal ,pants ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 4] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Home,Books,Electronics) AND i_class#3 IN (wallpaper,parenting,musical)) OR (i_category#4 IN (Shoes,Jewelry,Men) AND i_class#3 IN (womens,birdal,pants))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Home ,Books ,Electronics ) AND i_class#3 IN (wallpaper ,parenting ,musical )) OR (i_category#4 IN (Shoes ,Jewelry ,Men ) AND i_class#3 IN (womens ,birdal ,pants ))) AND isnotnull(i_item_sk#1)) (4) Scan parquet default.store_sales Output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt index 09e8dbc57e336..9d369b5d211fc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt @@ -57,7 +57,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((((isnotnull(d_date#6) AND (d_date#6 >= 11323)) AND (d_date#6 <= 11353)) AND (d_date_sk#5 >= 2451911)) AND (d_date_sk#5 <= 2451941)) AND isnotnull(d_date_sk#5)) +Condition : (((((isnotnull(d_date#6) AND (d_date#6 >= 2001-01-01)) AND (d_date#6 <= 2001-01-31)) AND (d_date_sk#5 >= 2451911)) AND (d_date_sk#5 <= 2451941)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Jewelry,Sports,Books]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Jewelry ,Sports ,Books ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -96,7 +96,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Jewelry,Sports,Books) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Jewelry ,Sports ,Books ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt index d46901132fcc3..e31c029b4932e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt @@ -46,7 +46,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Jewelry,Sports,Books]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Jewelry ,Sports ,Books ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -54,7 +54,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Jewelry,Sports,Books) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Jewelry ,Sports ,Books ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((((isnotnull(d_date#13) AND (d_date#13 >= 11323)) AND (d_date#13 <= 11353)) AND (d_date_sk#12 >= 2451911)) AND (d_date_sk#12 <= 2451941)) AND isnotnull(d_date_sk#12)) +Condition : (((((isnotnull(d_date#13) AND (d_date#13 >= 2001-01-01)) AND (d_date#13 <= 2001-01-31)) AND (d_date_sk#12 >= 2451911)) AND (d_date_sk#12 <= 2451941)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt index 4e258bb08b1d8..039a38acb9f22 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 0ae1f849bca42..0c22f6333e5a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(ws_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt index e400eea2cea66..55dca59270306 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt @@ -58,7 +58,7 @@ Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnot Output [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -66,7 +66,7 @@ Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] (6) Filter [codegen id : 1] Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] -Condition : (isnotnull(cd_demo_sk#12) AND ((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) OR ((cd_marital_status#13 = S) AND (cd_education_status#14 = College))) OR ((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)))) +Condition : (isnotnull(cd_demo_sk#12) AND ((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) OR ((cd_marital_status#13 = S) AND (cd_education_status#14 = College ))) OR ((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )))) (7) BroadcastExchange Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] @@ -75,7 +75,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (8) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#12] -Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) (9) Project [codegen id : 6] Output [11]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#13, cd_education_status#14] @@ -102,7 +102,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#16] -Join condition: (((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#17 = 3)) OR (((((cd_marital_status#13 = S) AND (cd_education_status#14 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#17 = 1))) OR (((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#17 = 1))) +Join condition: (((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#17 = 3)) OR (((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#17 = 1))) OR (((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#17 = 1))) (15) Project [codegen id : 6] Output [7]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index 560738591c930..027fde38ff7fa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -147,7 +147,7 @@ Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ex Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (25) ColumnarToRow [codegen id : 4] @@ -155,7 +155,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (26) Filter [codegen id : 4] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : (isnotnull(cd_demo_sk#21) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)))) +Condition : (isnotnull(cd_demo_sk#21) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College ))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )))) (27) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] @@ -164,7 +164,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (28) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#21] -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) (29) Project [codegen id : 6] Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] @@ -191,7 +191,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#25] -Join condition: (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#26 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#26 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#26 = 1))) +Join condition: (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#26 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#26 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#26 = 1))) (35) Project [codegen id : 6] Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index adfa3e44f996d..9c56438870f64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -203,7 +203,7 @@ Input [2]: [d_date_sk#22, d_date#23] (36) Filter [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11719)) AND (d_date#23 <= 11779)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2002-02-01)) AND (d_date#23 <= 2002-04-02)) AND isnotnull(d_date_sk#22)) (37) Project [codegen id : 10] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 5640564564396..dde6a9f564859 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -141,7 +141,7 @@ Input [2]: [d_date_sk#16, d_date#17] (22) Filter [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11719)) AND (d_date#17 <= 11779)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2002-02-01)) AND (d_date#17 <= 2002-04-02)) AND isnotnull(d_date_sk#16)) (23) Project [codegen id : 8] Output [1]: [d_date_sk#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index 97467fe1dc1db..8918c2a36e2ec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -77,7 +77,7 @@ Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isno Output [2]: [d_date_sk#8, d_quarter_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -85,7 +85,7 @@ Input [2]: [d_date_sk#8, d_quarter_name#9] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#8, d_quarter_name#9] -Condition : ((isnotnull(d_quarter_name#9) AND (d_quarter_name#9 = 2001Q1)) AND isnotnull(d_date_sk#8)) +Condition : ((isnotnull(d_quarter_name#9) AND (rpad(d_quarter_name#9, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#8)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#8] @@ -197,7 +197,7 @@ Condition : ((isnotnull(sr_customer_sk#21) AND isnotnull(sr_item_sk#20)) AND isn Output [2]: [d_date_sk#26, d_quarter_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] @@ -205,7 +205,7 @@ Input [2]: [d_date_sk#26, d_quarter_name#27] (33) Filter [codegen id : 9] Input [2]: [d_date_sk#26, d_quarter_name#27] -Condition : (d_quarter_name#27 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) +Condition : (rpad(d_quarter_name#27, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) (34) Project [codegen id : 9] Output [1]: [d_date_sk#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 32c8de9a1d16e..dafa24943be82 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -124,7 +124,7 @@ Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_i Output [2]: [d_date_sk#21, d_quarter_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 3] @@ -132,7 +132,7 @@ Input [2]: [d_date_sk#21, d_quarter_name#22] (18) Filter [codegen id : 3] Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : ((isnotnull(d_quarter_name#22) AND (d_quarter_name#22 = 2001Q1)) AND isnotnull(d_date_sk#21)) +Condition : ((isnotnull(d_quarter_name#22) AND (rpad(d_quarter_name#22, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#21)) (19) Project [codegen id : 3] Output [1]: [d_date_sk#21] @@ -155,7 +155,7 @@ Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_re Output [2]: [d_date_sk#24, d_quarter_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] @@ -163,7 +163,7 @@ Input [2]: [d_date_sk#24, d_quarter_name#25] (25) Filter [codegen id : 4] Input [2]: [d_date_sk#24, d_quarter_name#25] -Condition : (d_quarter_name#25 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) +Condition : (rpad(d_quarter_name#25, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) (26) Project [codegen id : 4] Output [1]: [d_date_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt index 020e92edb9d38..85aed12cc9d4f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt @@ -73,7 +73,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1) Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14 (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown)) AND isnotnull(cd_demo_sk#11)) +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] Output [2]: [cd_demo_sk#11, cd_dep_count#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index b3b8cabfb6880..ed68f03734b13 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -67,7 +67,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1) Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -75,7 +75,7 @@ Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14 (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown)) AND isnotnull(cd_demo_sk#11)) +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] Output [2]: [cd_demo_sk#11, cd_dep_count#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt index adb5346969d06..a40286f9f8f83 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index 1816433bdd129..4f5eebc0c300c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt index a04d4e6f5a5c2..efb45b5ccdb7f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt @@ -148,7 +148,7 @@ Results [4]: [w_warehouse_name#14, i_item_id#7, sum(CASE WHEN (d_date#11 < 2000- (26) Filter [codegen id : 5] Input [4]: [w_warehouse_name#14, i_item_id#7, inv_before#23, inv_after#24] -Condition : (((((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5)) +Condition : ((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (((inv_before#23 > 0) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5))) (27) TakeOrderedAndProject Input [4]: [w_warehouse_name#14, i_item_id#7, inv_before#23, inv_after#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 38d49852134cd..7fcbe1befa6b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -148,7 +148,7 @@ Results [4]: [w_warehouse_name#7, i_item_id#10, sum(CASE WHEN (d_date#14 < 2000- (26) Filter [codegen id : 5] Input [4]: [w_warehouse_name#7, i_item_id#10, inv_before#23, inv_after#24] -Condition : (((((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5)) +Condition : ((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (((inv_before#23 > 0) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5))) (27) TakeOrderedAndProject Input [4]: [w_warehouse_name#7, i_item_id#10, inv_before#23, inv_after#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index e9a0ada3801b4..431cf3ef92deb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -73,7 +73,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_ (7) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Condition : ((isnotnull(i_color#10) AND (i_color#10 = pale)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_color#10) AND (i_color#10 = pale )) AND isnotnull(i_item_sk#7)) (8) BroadcastExchange Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index 3b304ebe3f649..f73a5a5e052e3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -144,7 +144,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -152,7 +152,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] -Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale)) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale )) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 695f6efe560e4..163a4e7e3ddad 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -73,7 +73,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_ (7) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Condition : ((isnotnull(i_color#10) AND (i_color#10 = chiffon)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_color#10) AND (i_color#10 = chiffon )) AND isnotnull(i_item_sk#7)) (8) BroadcastExchange Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index 49f76525f88d8..09830b1be656e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -144,7 +144,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -152,7 +152,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] -Condition : ((isnotnull(i_color#21) AND (i_color#21 = chiffon)) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = chiffon )) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt index e38776ac2784b..40565dc76a7ba 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isn Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index 0de1f44569c19..c86f121ad35a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isn Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt index f4a07e6ac8f21..8c530f5a37885 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnu Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index f4a07e6ac8f21..8c530f5a37885 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnu Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index b70c36db4bc9d..01d0ca4a71e94 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 9537689459170..70575554e273e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -105,7 +105,7 @@ Input [2]: [d_date_sk#8, d_date#9] (16) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) (17) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index cba099c2bb5b0..1349aa54f3cc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -163,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index cba099c2bb5b0..1349aa54f3cc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -163,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt index 60b7965371e06..ea9994910c3b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index 6979de4765247..df5b93da51771 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt index 3f5ed7b48a9db..1f2c169ba0921 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt @@ -95,7 +95,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10988)) AND (d_date#12 <= 11048)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt index 04c3f07b903f0..9cc89345f5a1c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt @@ -92,7 +92,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10988)) AND (d_date#12 <= 11048)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt index 776585e5bd97f..71accc7295b0f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt @@ -177,11 +177,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (23) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] (24) Sort [codegen id : 8] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (25) Scan parquet default.catalog_sales Output [2]: [cs_bill_customer_sk#15, cs_sold_date_sk#16] @@ -254,15 +254,15 @@ Results [3]: [c_last_name#22, c_first_name#21, d_date#18] (40) Exchange Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] (41) Sort [codegen id : 16] Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 0) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 1970-01-01) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 (42) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18)] Join condition: None (43) HashAggregate [codegen id : 17] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (46) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] (47) Sort [codegen id : 19] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (48) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#27, ws_sold_date_sk#28] @@ -362,15 +362,15 @@ Results [3]: [c_last_name#34, c_first_name#33, d_date#30] (63) Exchange Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] +Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] (64) Sort [codegen id : 27] Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 0) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 1970-01-01) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 (65) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30)] Join condition: None (66) HashAggregate [codegen id : 28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index cedc6c7f4c576..c4e679eb902e0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -205,11 +205,11 @@ Results [3]: [c_last_name#19, c_first_name#18, d_date#16] (32) BroadcastExchange Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#21] (33) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 0), isnull(d_date#16)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] Join condition: None (34) HashAggregate [codegen id : 12] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#28, c_first_name#27, d_date#25] (48) BroadcastExchange Input [3]: [c_last_name#28, c_first_name#27, d_date#25] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#30] (49) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 0), isnull(d_date#25)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 1970-01-01), isnull(d_date#25)] Join condition: None (50) HashAggregate [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt index a53bec582637d..68ab7a8bbecab 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -280,7 +280,7 @@ Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_qu (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stdev#53, mean#54] -Condition : (((isnotnull(mean#54) AND isnotnull(stdev#53)) AND NOT (mean#54 = 0.0)) AND ((stdev#53 / mean#54) > 1.0)) +Condition : ((isnotnull(mean#54) AND isnotnull(stdev#53)) AND (NOT (mean#54 = 0.0) AND ((stdev#53 / mean#54) > 1.0))) (50) Project [codegen id : 11] Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, CASE WHEN (mean#54 = 0.0) THEN null ELSE (stdev#53 / mean#54) END AS cov#55] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index c61480a05a4cf..7abd3bb1acb11 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -173,7 +173,7 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -269,7 +269,7 @@ Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_qu (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] -Condition : (((isnotnull(mean#53) AND isnotnull(stdev#52)) AND NOT (mean#53 = 0.0)) AND ((stdev#52 / mean#53) > 1.0)) +Condition : ((isnotnull(mean#53) AND isnotnull(stdev#52)) AND (NOT (mean#53 = 0.0) AND ((stdev#52 / mean#53) > 1.0))) (48) Project [codegen id : 9] Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt index 6bc5980231252..f0dfbac4f3a0f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -280,7 +280,7 @@ Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_qu (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stdev#53, mean#54] -Condition : (((isnotnull(mean#54) AND isnotnull(stdev#53)) AND NOT (mean#54 = 0.0)) AND ((stdev#53 / mean#54) > 1.0)) +Condition : ((isnotnull(mean#54) AND isnotnull(stdev#53)) AND (NOT (mean#54 = 0.0) AND ((stdev#53 / mean#54) > 1.0))) (50) Project [codegen id : 11] Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, CASE WHEN (mean#54 = 0.0) THEN null ELSE (stdev#53 / mean#54) END AS cov#55] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 0ffe40240be2b..d1a5ecaa6a4d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -173,7 +173,7 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -269,7 +269,7 @@ Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_qu (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] -Condition : (((isnotnull(mean#53) AND isnotnull(stdev#52)) AND NOT (mean#53 = 0.0)) AND ((stdev#52 / mean#53) > 1.0)) +Condition : ((isnotnull(mean#53) AND isnotnull(stdev#52)) AND (NOT (mean#53 = 0.0) AND ((stdev#52 / mean#53) > 1.0))) (48) Project [codegen id : 9] Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index 13d73e61e1443..abb121a4c17ea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -66,7 +66,7 @@ Results [2]: [i_manufact#2, count#9] (10) Exchange Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] +Arguments: hashpartitioning(i_manufact#2, 5), ENSURE_REQUIREMENTS, [id=#10] (11) HashAggregate [codegen id : 2] Input [2]: [i_manufact#2, count#9] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 13d73e61e1443..abb121a4c17ea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -66,7 +66,7 @@ Results [2]: [i_manufact#2, count#9] (10) Exchange Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] +Arguments: hashpartitioning(i_manufact#2, 5), ENSURE_REQUIREMENTS, [id=#10] (11) HashAggregate [codegen id : 2] Input [2]: [i_manufact#2, count#9] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index 35924d2ffd9a9..fdb95bd5de9d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -95,7 +95,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 10] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) +Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) (14) Project [codegen id : 10] Output [2]: [item_sk#11, rnk#17] @@ -133,7 +133,7 @@ Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, (22) Filter [codegen id : 7] Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) +Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) (23) Project [codegen id : 7] Output [2]: [item_sk#21, rnk#25] @@ -141,7 +141,7 @@ Input [3]: [item_sk#21, rank_col#22, rnk#25] (24) BroadcastExchange Input [2]: [item_sk#21, rnk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#26] (25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index 4ca133ffd838d..f52ad2e63762e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -96,7 +96,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 4] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) +Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) (14) Project [codegen id : 4] Output [2]: [item_sk#11, rnk#17] @@ -138,7 +138,7 @@ Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, (23) Filter [codegen id : 8] Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) +Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) (24) Project [codegen id : 8] Output [2]: [item_sk#21, rnk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt index bd8d979c32d17..2f87af685f2e5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt @@ -79,7 +79,7 @@ Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_ Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree)),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -87,7 +87,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (12) Filter [codegen id : 2] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College)))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree ))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College )))) (13) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -96,7 +96,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) (15) Project [codegen id : 5] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index bd8d979c32d17..2f87af685f2e5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt @@ -79,7 +79,7 @@ Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_ Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree)),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -87,7 +87,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (12) Filter [codegen id : 2] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College)))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree ))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College )))) (13) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -96,7 +96,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) (15) Project [codegen id : 5] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 2d3dea5d84565..a754cef8f2962 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -160,7 +160,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 11192)) AND (d_date#26 <= 11206)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 2000-08-23)) AND (d_date#26 <= 2000-09-06)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index f6fe8bee89369..b3eb1f9592ff3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -130,7 +130,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11192)) AND (d_date#23 <= 11206)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt index 1c18ab278209b..e0cd0729e0e32 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 3c41db90ad1aa..3bf06577e3212 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index c8058eeee7ca1..85906beeff369 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -188,7 +188,7 @@ Input [4]: [sold_date_sk#12, customer_sk#13, item_sk#14, d_date_sk#21] Output [3]: [i_item_sk#25, i_class#26, i_category#27] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women), EqualTo(i_class,maternity), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 6] @@ -196,7 +196,7 @@ Input [3]: [i_item_sk#25, i_class#26, i_category#27] (29) Filter [codegen id : 6] Input [3]: [i_item_sk#25, i_class#26, i_category#27] -Condition : ((((isnotnull(i_category#27) AND isnotnull(i_class#26)) AND (i_category#27 = Women)) AND (i_class#26 = maternity)) AND isnotnull(i_item_sk#25)) +Condition : ((((isnotnull(i_category#27) AND isnotnull(i_class#26)) AND (i_category#27 = Women )) AND (i_class#26 = maternity )) AND isnotnull(i_item_sk#25)) (30) Project [codegen id : 6] Output [1]: [i_item_sk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index b14ae55d75093..2c1f7c6195ba7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -109,7 +109,7 @@ Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] Output [3]: [i_item_sk#14, i_class#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women), EqualTo(i_class,maternity), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] ReadSchema: struct (11) ColumnarToRow [codegen id : 3] @@ -117,7 +117,7 @@ Input [3]: [i_item_sk#14, i_class#15, i_category#16] (12) Filter [codegen id : 3] Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women)) AND (i_class#15 = maternity)) AND isnotnull(i_item_sk#14)) +Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women )) AND (i_class#15 = maternity )) AND isnotnull(i_item_sk#14)) (13) Project [codegen id : 3] Output [1]: [i_item_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index ee3abb39ed053..f6dd5a59de4d4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -467,7 +467,7 @@ Input [2]: [d_date#5, d_week_seq#6] (82) Filter [codegen id : 1] Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_date#5) AND (d_date#5 = 10959)) +Condition : (isnotnull(d_date#5) AND (d_date#5 = 2000-01-03)) (83) Project [codegen id : 1] Output [1]: [d_week_seq#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index dd37b8801e957..ab4c4ad4ae65b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -467,7 +467,7 @@ Input [2]: [d_date#8, d_week_seq#9] (82) Filter [codegen id : 1] Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_date#8) AND (d_date#8 = 10959)) +Condition : (isnotnull(d_date#8) AND (d_date#8 = 2000-01-03)) (83) Project [codegen id : 1] Output [1]: [d_week_seq#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt index 3533986d86b84..2b8dbaddacad8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,refernece,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,refernece ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,refernece,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,refernece ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index 622b073a4ae10..efb27d8cd8b11 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,refernece,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,refernece ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,refernece,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,refernece ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt index 5d74065179f44..bd8dc07f42967 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt @@ -741,7 +741,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (121) ColumnarToRow [codegen id : 40] @@ -749,7 +749,7 @@ Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] (122) Filter [codegen id : 40] Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] -Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) +Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) (123) Project [codegen id : 40] Output [2]: [i_item_sk#93, i_product_name#96] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index 9cc3415e6e146..216e7f5530128 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -651,7 +651,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 24] @@ -659,7 +659,7 @@ Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] (106) Filter [codegen id : 24] Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] -Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) +Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) (107) Project [codegen id : 24] Output [2]: [i_item_sk#87, i_product_name#90] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt index 075fd1864222c..3a19c3923737d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt @@ -183,7 +183,7 @@ Arguments: [rank(sumsales#38) windowspecdefinition(i_category#23, sumsales#38 DE (33) Filter [codegen id : 10] Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#38, rk#40] -Condition : (isnotnull(rk#40) AND (rk#40 <= 100)) +Condition : (rk#40 <= 100) (34) TakeOrderedAndProject Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#38, rk#40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index ba71667cfebe5..3d42f49415640 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -168,7 +168,7 @@ Arguments: [rank(sumsales#37) windowspecdefinition(i_category#22, sumsales#37 DE (30) Filter [codegen id : 7] Input [10]: [i_category#22, i_class#23, i_brand#24, i_product_name#25, d_year#26, d_qoy#27, d_moy#28, s_store_id#29, sumsales#37, rk#39] -Condition : (isnotnull(rk#39) AND (rk#39 <= 100)) +Condition : (rk#39 <= 100) (31) TakeOrderedAndProject Input [10]: [i_category#22, i_class#23, i_brand#24, i_product_name#25, d_year#26, d_qoy#27, d_moy#28, s_store_id#29, sumsales#37, rk#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt index ab2cc784a5f22..b138d059eaecb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index 9c914d56f46a2..5f6002b84f411 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index d31dbc3498ead..9e2d33f2bca19 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : (ranking#19 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 26fec145f4211..cd12e362c44f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : (ranking#19 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt index 3daa8b66851f7..cd88a84ede266 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt @@ -180,7 +180,7 @@ Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] @@ -188,7 +188,7 @@ Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (33) Filter [codegen id : 8] Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) +Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) (34) Project [codegen id : 8] Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 3daa8b66851f7..cd88a84ede266 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -180,7 +180,7 @@ Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] @@ -188,7 +188,7 @@ Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (33) Filter [codegen id : 8] Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) +Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) (34) Project [codegen id : 8] Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index 04172ab353f25..339dc5d95e4c4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -100,7 +100,7 @@ Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotn Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -108,7 +108,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000)) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index fe49873e2e3f3..177b8e681608a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -207,7 +207,7 @@ Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_ Output [2]: [hd_demo_sk#24, hd_buy_potential#25] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -215,7 +215,7 @@ Input [2]: [hd_demo_sk#24, hd_buy_potential#25] (31) Filter [codegen id : 5] Input [2]: [hd_demo_sk#24, hd_buy_potential#25] -Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = >10000)) AND isnotnull(hd_demo_sk#24)) +Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = >10000 )) AND isnotnull(hd_demo_sk#24)) (32) Project [codegen id : 5] Output [1]: [hd_demo_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt index e658e11ade4a0..aec6d66c98fdd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index b45f0513ba576..646a8fbc11a3a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt index 9472a8935eb8b..f87f822166e23 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 9472a8935eb8b..f87f822166e23 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt index 6e86260f0aba8..e5c59a0ecfaad 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt @@ -244,7 +244,7 @@ Input [2]: [d_date_sk#22, d_date#23] (30) Filter [codegen id : 7] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11192)) AND (d_date#23 <= 11222)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-22)) AND isnotnull(d_date_sk#22)) (31) Project [codegen id : 7] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index 87abdc02b21a7..3d087efe472bb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -182,7 +182,7 @@ Input [2]: [d_date_sk#16, d_date#17] (16) Filter [codegen id : 5] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11192)) AND (d_date#17 <= 11222)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-08-23)) AND (d_date#17 <= 2000-09-22)) AND isnotnull(d_date_sk#16)) (17) Project [codegen id : 5] Output [1]: [d_date_sk#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt index 8b09336f86535..6725e273a3acc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt @@ -95,7 +95,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 11102)) AND (d_date#12 <= 11162)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt index 340fcf5fa31fd..a03333f7623cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt @@ -92,7 +92,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 11102)) AND (d_date#12 <= 11162)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt index 0073fd978d748..7f7c9ac72707e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt @@ -245,7 +245,7 @@ Arguments: [wr_refunded_cdemo_sk#13 ASC NULLS FIRST, wr_returning_cdemo_sk#15 AS Output [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (43) ColumnarToRow [codegen id : 11] @@ -253,7 +253,7 @@ Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] (44) Filter [codegen id : 11] Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] -Condition : (((isnotnull(cd_demo_sk#33) AND isnotnull(cd_marital_status#34)) AND isnotnull(cd_education_status#35)) AND ((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree)) OR ((cd_marital_status#34 = S) AND (cd_education_status#35 = College))) OR ((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree)))) +Condition : (((isnotnull(cd_demo_sk#33) AND isnotnull(cd_marital_status#34)) AND isnotnull(cd_education_status#35)) AND ((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree )) OR ((cd_marital_status#34 = S) AND (cd_education_status#35 = College ))) OR ((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree )))) (45) BroadcastExchange Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] @@ -293,7 +293,7 @@ Arguments: [cast(cd_demo_sk#33 as bigint) ASC NULLS FIRST, cast(cd_demo_sk#37 as (53) SortMergeJoin [codegen id : 14] Left keys [2]: [wr_refunded_cdemo_sk#13, wr_returning_cdemo_sk#15] Right keys [2]: [cast(cd_demo_sk#33 as bigint), cast(cd_demo_sk#37 as bigint)] -Join condition: ((((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree)) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#34 = S) AND (cd_education_status#35 = College)) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree)) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#34 = S) AND (cd_education_status#35 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) (54) Project [codegen id : 14] Output [4]: [ws_quantity#4, wr_fee#18, wr_refunded_cash#19, r_reason_desc#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index c528cf8f91a55..74a9aa3dec985 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -130,7 +130,7 @@ Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -138,7 +138,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (19) Filter [codegen id : 3] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : (((isnotnull(cd_demo_sk#21) AND isnotnull(cd_marital_status#22)) AND isnotnull(cd_education_status#23)) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)))) +Condition : (((isnotnull(cd_demo_sk#21) AND isnotnull(cd_marital_status#22)) AND isnotnull(cd_education_status#23)) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College ))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )))) (20) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] @@ -147,7 +147,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [wr_refunded_cdemo_sk#11] Right keys [1]: [cast(cd_demo_sk#21 as bigint)] -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) (22) Project [codegen id : 8] Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, cd_marital_status#22, cd_education_status#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt index 888fd8e520796..92895cb566fd2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt @@ -177,11 +177,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (23) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] (24) Sort [codegen id : 8] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (25) Scan parquet default.catalog_sales Output [2]: [cs_bill_customer_sk#15, cs_sold_date_sk#16] @@ -254,15 +254,15 @@ Results [3]: [c_last_name#22, c_first_name#21, d_date#18] (40) Exchange Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] (41) Sort [codegen id : 16] Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 0) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 1970-01-01) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 (42) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18)] Join condition: None (43) HashAggregate [codegen id : 17] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (46) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] (47) Sort [codegen id : 19] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (48) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#27, ws_sold_date_sk#28] @@ -362,15 +362,15 @@ Results [3]: [c_last_name#34, c_first_name#33, d_date#30] (63) Exchange Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] +Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] (64) Sort [codegen id : 27] Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 0) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 1970-01-01) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 (65) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30)] Join condition: None (66) HashAggregate [codegen id : 28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index bc62b67957935..27e16b75638a8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -205,11 +205,11 @@ Results [3]: [c_last_name#19, c_first_name#18, d_date#16] (32) BroadcastExchange Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#21] (33) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 0), isnull(d_date#16)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] Join condition: None (34) HashAggregate [codegen id : 12] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#28, c_first_name#27, d_date#25] (48) BroadcastExchange Input [3]: [c_last_name#28, c_first_name#27, d_date#25] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#30] (49) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 0), isnull(d_date#25)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 1970-01-01), isnull(d_date#25)] Join condition: None (50) HashAggregate [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt index c9ebf0e9c9426..b3d0c07cd80fe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Books,Electronics,Sports]),In(i_class, [computers,stereo,football])),And(In(i_category, [Men,Jewelry,Women]),In(i_class, [shirts,birdal,dresses]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Books ,Electronics ,Sports ]),In(i_class, [computers ,stereo ,football ])),And(In(i_category, [Men ,Jewelry ,Women ]),In(i_class, [shirts ,birdal ,dresses ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 1] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Books,Electronics,Sports) AND i_class#3 IN (computers,stereo,football)) OR (i_category#4 IN (Men,Jewelry,Women) AND i_class#3 IN (shirts,birdal,dresses))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Books ,Electronics ,Sports ) AND i_class#3 IN (computers ,stereo ,football )) OR (i_category#4 IN (Men ,Jewelry ,Women ) AND i_class#3 IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) (4) BroadcastExchange Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index cbf6273e28568..f61c8e6945003 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Books,Electronics,Sports]),In(i_class, [computers,stereo,football])),And(In(i_category, [Men,Jewelry,Women]),In(i_class, [shirts,birdal,dresses]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Books ,Electronics ,Sports ]),In(i_class, [computers ,stereo ,football ])),And(In(i_category, [Men ,Jewelry ,Women ]),In(i_class, [shirts ,birdal ,dresses ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 4] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Books,Electronics,Sports) AND i_class#3 IN (computers,stereo,football)) OR (i_category#4 IN (Men,Jewelry,Women) AND i_class#3 IN (shirts,birdal,dresses))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Books ,Electronics ,Sports ) AND i_class#3 IN (computers ,stereo ,football )) OR (i_category#4 IN (Men ,Jewelry ,Women ) AND i_class#3 IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) (4) Scan parquet default.store_sales Output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt index e38c455eb6be2..3554ce5dcb117 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt @@ -187,7 +187,7 @@ Input [5]: [cr_call_center_sk#2, cr_net_loss#3, c_current_cdemo_sk#12, c_current Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown )),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree ))), IsNotNull(cd_demo_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 5] @@ -195,7 +195,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (33) Filter [codegen id : 5] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Unknown)) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = Advanced Degree))) AND isnotnull(cd_demo_sk#21)) +Condition : ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Unknown )) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = Advanced Degree ))) AND isnotnull(cd_demo_sk#21)) (34) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index c14a8421bc3b0..fc0da4adf7b16 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -183,7 +183,7 @@ Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_curre Output [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown )),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree ))), IsNotNull(cd_demo_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 5] @@ -191,7 +191,7 @@ Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] (32) Filter [codegen id : 5] Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Condition : ((((cd_marital_status#24 = M) AND (cd_education_status#25 = Unknown)) OR ((cd_marital_status#24 = W) AND (cd_education_status#25 = Advanced Degree))) AND isnotnull(cd_demo_sk#23)) +Condition : ((((cd_marital_status#24 = M) AND (cd_education_status#25 = Unknown )) OR ((cd_marital_status#24 = W) AND (cd_education_status#25 = Advanced Degree ))) AND isnotnull(cd_demo_sk#23)) (33) BroadcastExchange Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index 8297114e1e9ab..1fdf74c2ce164 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -84,7 +84,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index faf82026138f6..06c28e176aa1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -108,7 +108,7 @@ Input [2]: [d_date_sk#8, d_date#9] (16) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) (17) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index aec81c0d5c7aa..2466a4d38f21e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -47,7 +47,7 @@ Input [5]: [sr_item_sk#1, sr_reason_sk#2, sr_ticket_number#3, sr_return_quantity Output [2]: [r_reason_sk#6, r_reason_desc#7] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -55,7 +55,7 @@ Input [2]: [r_reason_sk#6, r_reason_desc#7] (7) Filter [codegen id : 1] Input [2]: [r_reason_sk#6, r_reason_desc#7] -Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28)) AND isnotnull(r_reason_sk#6)) +Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28 )) AND isnotnull(r_reason_sk#6)) (8) Project [codegen id : 1] Output [1]: [r_reason_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 1875c5418a890..1f3470198cd20 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -85,7 +85,7 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, s Output [2]: [r_reason_sk#14, r_reason_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 5] @@ -93,7 +93,7 @@ Input [2]: [r_reason_sk#14, r_reason_desc#15] (16) Filter [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] -Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28)) AND isnotnull(r_reason_sk#14)) +Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28 )) AND isnotnull(r_reason_sk#14)) (17) Project [codegen id : 5] Output [1]: [r_reason_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index bee7110ecd6dd..c77590bf71044 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -175,7 +175,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Output [2]: [web_site_sk#20, web_company_name#21] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 10] @@ -183,7 +183,7 @@ Input [2]: [web_site_sk#20, web_company_name#21] (31) Filter [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] -Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri)) AND isnotnull(web_site_sk#20)) +Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri )) AND isnotnull(web_site_sk#20)) (32) Project [codegen id : 10] Output [1]: [web_site_sk#20] @@ -214,7 +214,7 @@ Input [2]: [d_date_sk#23, d_date#24] (38) Filter [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] -Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 10623)) AND (d_date#24 <= 10683)) AND isnotnull(d_date_sk#23)) +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 1999-02-01)) AND (d_date#24 <= 1999-04-02)) AND isnotnull(d_date_sk#23)) (39) Project [codegen id : 11] Output [1]: [d_date_sk#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index efa09c2f625cd..9558a01423452 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -152,7 +152,7 @@ Input [2]: [d_date_sk#17, d_date#18] (24) Filter [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10623)) AND (d_date#18 <= 10683)) AND isnotnull(d_date_sk#17)) +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-01)) AND (d_date#18 <= 1999-04-02)) AND isnotnull(d_date_sk#17)) (25) Project [codegen id : 9] Output [1]: [d_date_sk#17] @@ -206,7 +206,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_ Output [2]: [web_site_sk#23, web_company_name#24] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] @@ -214,7 +214,7 @@ Input [2]: [web_site_sk#23, web_company_name#24] (38) Filter [codegen id : 11] Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri)) AND isnotnull(web_site_sk#23)) +Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) (39) Project [codegen id : 11] Output [1]: [web_site_sk#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index 725939ef24f1b..320a93e19bb27 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -252,7 +252,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Output [2]: [web_site_sk#24, web_company_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 19] @@ -260,7 +260,7 @@ Input [2]: [web_site_sk#24, web_company_name#25] (46) Filter [codegen id : 19] Input [2]: [web_site_sk#24, web_company_name#25] -Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri)) AND isnotnull(web_site_sk#24)) +Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri )) AND isnotnull(web_site_sk#24)) (47) Project [codegen id : 19] Output [1]: [web_site_sk#24] @@ -291,7 +291,7 @@ Input [2]: [d_date_sk#27, d_date#28] (53) Filter [codegen id : 20] Input [2]: [d_date_sk#27, d_date#28] -Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 10623)) AND (d_date#28 <= 10683)) AND isnotnull(d_date_sk#27)) +Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 1999-02-01)) AND (d_date#28 <= 1999-04-02)) AND isnotnull(d_date_sk#27)) (54) Project [codegen id : 20] Output [1]: [d_date_sk#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 9f15375b5cfc0..2f719a4be1630 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -229,7 +229,7 @@ Input [2]: [d_date_sk#21, d_date#22] (39) Filter [codegen id : 18] Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 10623)) AND (d_date#22 <= 10683)) AND isnotnull(d_date_sk#21)) +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-01)) AND (d_date#22 <= 1999-04-02)) AND isnotnull(d_date_sk#21)) (40) Project [codegen id : 18] Output [1]: [d_date_sk#21] @@ -283,7 +283,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_ Output [2]: [web_site_sk#27, web_company_name#28] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 20] @@ -291,7 +291,7 @@ Input [2]: [web_site_sk#27, web_company_name#28] (53) Filter [codegen id : 20] Input [2]: [web_site_sk#27, web_company_name#28] -Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri)) AND isnotnull(web_site_sk#27)) +Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) (54) Project [codegen id : 20] Output [1]: [web_site_sk#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt index 7b8e4ff5516a7..98c1bc5671cd9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt @@ -57,7 +57,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -96,7 +96,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 3457f398b7dea..1507b5ccbc0ae 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -46,7 +46,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -54,7 +54,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt index 15c445b6b1325..dd85594160c8f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index 774081bc5c3f6..f6dd412f5c4b1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(ws_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt index 8bc09f9457fe7..a74ad3e4b8b77 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index 49b35aacfcf2f..c6a345be29c9d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt index d4d0015a9fec0..8031170978906 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt @@ -157,7 +157,7 @@ Input [12]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk# Output [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 4] @@ -165,7 +165,7 @@ Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, (26) Filter [codegen id : 4] Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Condition : ((isnotnull(i_color#27) AND (i_color#27 = pale)) AND isnotnull(i_item_sk#24)) +Condition : ((isnotnull(i_color#27) AND (i_color#27 = pale )) AND isnotnull(i_item_sk#24)) (27) BroadcastExchange Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index 6835ff84f5bc7..540feb0d88a3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -146,7 +146,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -154,7 +154,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] -Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale)) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale )) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt index 104340f2fd6db..f6151f4d79ec1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt index f106fba199355..e4d14f842f5a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index e04148fad545e..5374a708a0295 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -173,7 +173,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 10442)) AND (d_date#26 <= 10456)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 1998-08-04)) AND (d_date#26 <= 1998-08-18)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 36b1ff63b2065..89362fb4f0efe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -143,7 +143,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 10442)) AND (d_date#23 <= 10456)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt index c096a28386361..2a9e1ad7c715b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt @@ -741,7 +741,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (121) ColumnarToRow [codegen id : 40] @@ -749,7 +749,7 @@ Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] (122) Filter [codegen id : 40] Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] -Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) +Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) (123) Project [codegen id : 40] Output [2]: [i_item_sk#93, i_product_name#96] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 9424b7f1e2225..5c2ac1abaaa7e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -651,7 +651,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 24] @@ -659,7 +659,7 @@ Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] (106) Filter [codegen id : 24] Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] -Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) +Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) (107) Project [codegen id : 24] Output [2]: [i_item_sk#87, i_product_name#90] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt index 6d9adf1d38106..413a918da6a0f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt @@ -445,7 +445,7 @@ Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DE (73) Filter [codegen id : 82] Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] -Condition : (isnotnull(rk#148) AND (rk#148 <= 100)) +Condition : (rk#148 <= 100) (74) TakeOrderedAndProject Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index ae6b3ff4d7542..fa822f3ac9ed7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -430,7 +430,7 @@ Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DE (70) Filter [codegen id : 55] Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] -Condition : (isnotnull(rk#147) AND (rk#147 <= 100)) +Condition : (rk#147 <= 100) (71) TakeOrderedAndProject Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt index 3c65529504320..e024d06c710a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : (ranking#19 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 406acb0e0a27f..5ae5fd82839cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : (ranking#19 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index 0d90342fc9bbf..fe8ceb415f571 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -100,7 +100,7 @@ Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotn Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -108,7 +108,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000)) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index c897c78880bc2..0e67565e856ad 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -207,7 +207,7 @@ Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_ Output [2]: [hd_demo_sk#24, hd_buy_potential#25] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -215,7 +215,7 @@ Input [2]: [hd_demo_sk#24, hd_buy_potential#25] (31) Filter [codegen id : 5] Input [2]: [hd_demo_sk#24, hd_buy_potential#25] -Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = 1001-5000)) AND isnotnull(hd_demo_sk#24)) +Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = 1001-5000 )) AND isnotnull(hd_demo_sk#24)) (32) Project [codegen id : 5] Output [1]: [hd_demo_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt index 955dde96eb177..7ecbe2e51c9b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 955dde96eb177..7ecbe2e51c9b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt index e192ab8d637de..8a384ec4b1795 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt @@ -257,7 +257,7 @@ Input [2]: [d_date_sk#22, d_date#23] (30) Filter [codegen id : 7] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 10442)) AND (d_date#23 <= 10472)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-09-03)) AND isnotnull(d_date_sk#22)) (31) Project [codegen id : 7] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index e027ef7e53e8a..596f9497b5905 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt @@ -195,7 +195,7 @@ Input [2]: [d_date_sk#16, d_date#17] (16) Filter [codegen id : 5] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 10442)) AND (d_date#17 <= 10472)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 1998-08-04)) AND (d_date#17 <= 1998-09-03)) AND isnotnull(d_date_sk#16)) (17) Project [codegen id : 5] Output [1]: [d_date_sk#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt index b036ff059dbb2..89bb3e5d551a8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt @@ -56,7 +56,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -87,7 +87,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -95,7 +95,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index 920a4c2a55a26..b7c5f4081a1f3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -45,7 +45,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -53,7 +53,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -80,7 +80,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index 76204c504c0ed..518597f59690d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -44,9 +44,6 @@ import org.apache.spark.sql.internal.SQLConf * actual explain plan: /path/to/tmp/q1.actual.explain.txt * [actual simplified plan] * - * The explain files are saved to help debug later, they are not checked. Only the simplified - * plans are checked (by string comparison). - * * * To run the entire test suite: * {{{ @@ -101,10 +98,13 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite { new File(goldenFilePath, name) } - private def isApproved(dir: File, actualSimplifiedPlan: String): Boolean = { - val file = new File(dir, "simplified.txt") - val expected = FileUtils.readFileToString(file, StandardCharsets.UTF_8) - expected == actualSimplifiedPlan + private def isApproved( + dir: File, actualSimplifiedPlan: String, actualExplain: String): Boolean = { + val simplifiedFile = new File(dir, "simplified.txt") + val expectedSimplified = FileUtils.readFileToString(simplifiedFile, StandardCharsets.UTF_8) + lazy val explainFile = new File(dir, "explain.txt") + lazy val expectedExplain = FileUtils.readFileToString(explainFile, StandardCharsets.UTF_8) + expectedSimplified == actualSimplifiedPlan && expectedExplain == actualExplain } /** @@ -119,7 +119,7 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite { private def generateGoldenFile(plan: SparkPlan, name: String, explain: String): Unit = { val dir = getDirForTest(name) val simplified = getSimplifiedPlan(plan) - val foundMatch = dir.exists() && isApproved(dir, simplified) + val foundMatch = dir.exists() && isApproved(dir, simplified, explain) if (!foundMatch) { FileUtils.deleteDirectory(dir) @@ -137,7 +137,7 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite { val dir = getDirForTest(name) val tempDir = FileUtils.getTempDirectory val actualSimplified = getSimplifiedPlan(plan) - val foundMatch = isApproved(dir, actualSimplified) + val foundMatch = isApproved(dir, actualSimplified, explain) if (!foundMatch) { // show diff with last approved From abfd9b23cd7c21e9525df85a16e0611ef0f35908 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 24 Mar 2021 15:04:03 +0000 Subject: [PATCH 20/24] [SPARK-34769][SQL] AnsiTypeCoercion: return closest convertible type among TypeCollection ### What changes were proposed in this pull request? Currently, when implicit casting a data type to a `TypeCollection`, Spark returns the first convertible data type among `TypeCollection`. In ANSI mode, we can make the behavior more reasonable by returning the closet convertible data type in `TypeCollection`. In details, we first try to find the all the expected types we can implicitly cast: 1. if there is no convertible data types, return None; 2. if there is only one convertible data type, cast input as it; 3. otherwise if there are multiple convertible data types, find the closet data type among them. If there is no such closet data type, return None. Note that if the closet type is Float type and the convertible types contains Double type, simply return Double type as the closet type to avoid potential precision loss on converting the Integral type as Float type. ### Why are the changes needed? Make the type coercion rule for TypeCollection more reasonable and ANSI compatible. E.g. returning Long instead of Double for`implicast(int, TypeCollect(Double, Long))`. From ANSI SQL Spec section 4.33 "SQL-invoked routines" ![Screen Shot 2021-03-17 at 4 05 06 PM](https://user-images.githubusercontent.com/1097932/111434916-5e104e80-86bd-11eb-8b3b-33090a68067d.png) Section 9.6 "Subject routine determination" ![Screen Shot 2021-03-17 at 1 36 55 PM](https://user-images.githubusercontent.com/1097932/111420336-48445e80-86a8-11eb-9d50-34b325043bdb.png) Section 10.4 "routine invocation" ![Screen Shot 2021-03-17 at 4 08 41 PM](https://user-images.githubusercontent.com/1097932/111434926-610b3f00-86bd-11eb-8c32-8c7935e055da.png) ### Does this PR introduce _any_ user-facing change? Yes, in ANSI mode, implicit casting to a `TypeCollection` returns the narrowest convertible data type instead of the first convertible one. ### How was this patch tested? Unit tests. Closes #31859 from gengliangwang/implicitCastTypeCollection. Lead-authored-by: Gengliang Wang Co-authored-by: Gengliang Wang Signed-off-by: Wenchen Fan --- .../catalyst/analysis/AnsiTypeCoercion.scala | 35 +++++++++++++++++-- .../analysis/AnsiTypeCoercionSuite.scala | 31 +++++++++++----- .../sql-tests/inputs/string-functions.sql | 6 ++-- .../results/ansi/string-functions.sql.out | 26 +++++++++++--- .../sql-tests/results/postgreSQL/text.sql.out | 6 ++-- .../results/string-functions.sql.out | 30 ++++++++++++---- 6 files changed, 106 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala index b6799015313ce..4c31b52d62b4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala @@ -158,7 +158,10 @@ object AnsiTypeCoercion extends TypeCoercionBase { case _ if expectedType.acceptsType(inType) => Some(inType) // Cast null type (usually from null literals) into target types - case (NullType, target) => Some(target.defaultConcreteType) + // By default, the result type is `target.defaultConcreteType`. When the target type is + // `TypeCollection`, there is another branch to find the "closet convertible data type" below. + case (NullType, target) if !target.isInstanceOf[TypeCollection] => + Some(target.defaultConcreteType) // This type coercion system will allow implicit converting String type literals as other // primitive types, in case of breaking too many existing Spark SQL queries. @@ -191,9 +194,35 @@ object AnsiTypeCoercion extends TypeCoercionBase { case (DateType, TimestampType) => Some(TimestampType) // When we reach here, input type is not acceptable for any types in this type collection, - // try to find the first one we can implicitly cast. + // first try to find the all the expected types we can implicitly cast: + // 1. if there is no convertible data types, return None; + // 2. if there is only one convertible data type, cast input as it; + // 3. otherwise if there are multiple convertible data types, find the closet convertible + // data type among them. If there is no such a data type, return None. case (_, TypeCollection(types)) => - types.flatMap(implicitCast(inType, _, isInputFoldable)).headOption + // Since Spark contains special objects like `NumericType` and `DecimalType`, which accepts + // multiple types and they are `AbstractDataType` instead of `DataType`, here we use the + // conversion result their representation. + val convertibleTypes = types.flatMap(implicitCast(inType, _, isInputFoldable)) + if (convertibleTypes.isEmpty) { + None + } else { + // find the closet convertible data type, which can be implicit cast to all other + // convertible types. + val closestConvertibleType = convertibleTypes.find { dt => + convertibleTypes.forall { target => + implicitCast(dt, target, isInputFoldable = false).isDefined + } + } + // If the closet convertible type is Float type and the convertible types contains Double + // type, simply return Double type as the closet convertible type to avoid potential + // precision loss on converting the Integral type as Float type. + if (closestConvertibleType.contains(FloatType) && convertibleTypes.contains(DoubleType)) { + Some(DoubleType) + } else { + closestConvertibleType + } + } // Implicit cast between array types. // diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala index 88e082f1580ef..e3e61f022c7b3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala @@ -345,8 +345,6 @@ class AnsiTypeCoercionSuite extends AnalysisTest { } test("eligible implicit type cast - TypeCollection") { - shouldCast(NullType, TypeCollection(StringType, BinaryType), StringType) - shouldCast(StringType, TypeCollection(StringType, BinaryType), StringType) shouldCast(BinaryType, TypeCollection(StringType, BinaryType), BinaryType) shouldCast(StringType, TypeCollection(BinaryType, StringType), StringType) @@ -356,17 +354,10 @@ class AnsiTypeCoercionSuite extends AnalysisTest { shouldCast(BinaryType, TypeCollection(BinaryType, IntegerType), BinaryType) shouldCast(BinaryType, TypeCollection(IntegerType, BinaryType), BinaryType) - shouldNotCast(IntegerType, TypeCollection(StringType, BinaryType)) - shouldNotCast(IntegerType, TypeCollection(BinaryType, StringType)) - shouldCast(DecimalType.SYSTEM_DEFAULT, TypeCollection(IntegerType, DecimalType), DecimalType.SYSTEM_DEFAULT) shouldCast(DecimalType(10, 2), TypeCollection(IntegerType, DecimalType), DecimalType(10, 2)) shouldCast(DecimalType(10, 2), TypeCollection(DecimalType, IntegerType), DecimalType(10, 2)) - shouldNotCast(IntegerType, TypeCollection(DecimalType(10, 2), StringType)) - - shouldNotCastStringInput(TypeCollection(NumericType, BinaryType)) - shouldCastStringLiteral(TypeCollection(NumericType, BinaryType), DoubleType) shouldCast( ArrayType(StringType, false), @@ -377,10 +368,32 @@ class AnsiTypeCoercionSuite extends AnalysisTest { ArrayType(StringType, true), TypeCollection(ArrayType(StringType), StringType), ArrayType(StringType, true)) + + // When there are multiple convertible types in the `TypeCollection`, use the closest + // convertible data type among convertible types. + shouldCast(IntegerType, TypeCollection(BinaryType, FloatType, LongType), LongType) + shouldCast(ShortType, TypeCollection(BinaryType, LongType, IntegerType), IntegerType) + shouldCast(ShortType, TypeCollection(DateType, LongType, IntegerType, DoubleType), IntegerType) + // If the result is Float type and Double type is also among the convertible target types, + // use Double Type instead of Float type. + shouldCast(LongType, TypeCollection(FloatType, DoubleType, StringType), DoubleType) } test("ineligible implicit type cast - TypeCollection") { + shouldNotCast(IntegerType, TypeCollection(StringType, BinaryType)) + shouldNotCast(IntegerType, TypeCollection(BinaryType, StringType)) shouldNotCast(IntegerType, TypeCollection(DateType, TimestampType)) + shouldNotCast(IntegerType, TypeCollection(DecimalType(10, 2), StringType)) + shouldNotCastStringInput(TypeCollection(NumericType, BinaryType)) + // When there are multiple convertible types in the `TypeCollection` and there is no such + // a data type that can be implicit cast to all the other convertible types in the collection. + Seq(TypeCollection(NumericType, BinaryType), + TypeCollection(NumericType, DecimalType, BinaryType), + TypeCollection(IntegerType, LongType, BooleanType), + TypeCollection(DateType, TimestampType, BooleanType)).foreach { typeCollection => + shouldNotCastStringLiteral(typeCollection) + shouldNotCast(NullType, typeCollection) + } } test("tightest common bound for types") { diff --git a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql index 47c1a6debb72e..d44055d72e3bc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql @@ -17,9 +17,11 @@ select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aa -- left && right select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null); -select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a'); +select left(null, -2); +select left("abcd", -2), left("abcd", 0), left("abcd", 'a'); select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null); -select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a'); +select right(null, -2); +select right("abcd", -2), right("abcd", 0), right("abcd", 'a'); -- split function SELECT split('aa1cc2ee3', '[1-9]+'); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index a1f1d87f5a594..3f4399fe088b3 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 48 +-- Number of queries: 50 -- !query @@ -69,7 +69,16 @@ ab abcd ab NULL -- !query -select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a') +select left(null, -2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'substring(NULL, 1, -2)' due to data type mismatch: argument 1 requires (string or binary) type, however, 'NULL' is of null type.; line 1 pos 7 + + +-- !query +select left("abcd", -2), left("abcd", 0), left("abcd", 'a') -- !query schema struct<> -- !query output @@ -87,12 +96,21 @@ cannot resolve 'substring('abcd', (- CAST('2' AS DOUBLE)), 2147483647)' due to d -- !query -select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a') +select right(null, -2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'substring(NULL, (- -2), 2147483647)' due to data type mismatch: argument 1 requires (string or binary) type, however, 'NULL' is of null type.; line 1 pos 7 + + +-- !query +select right("abcd", -2), right("abcd", 0), right("abcd", 'a') -- !query schema struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve 'substring('abcd', (- CAST('a' AS DOUBLE)), 2147483647)' due to data type mismatch: argument 2 requires int type, however, '(- CAST('a' AS DOUBLE))' is of double type.; line 1 pos 61 +cannot resolve 'substring('abcd', (- CAST('a' AS DOUBLE)), 2147483647)' due to data type mismatch: argument 2 requires int type, however, '(- CAST('a' AS DOUBLE))' is of double type.; line 1 pos 44 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index 0ecba2d11acc4..2387dd244181b 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -130,7 +130,7 @@ select concat_ws(',',10,20,null,30) struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve 'concat_ws(',', 10, 20, NULL, 30)' due to data type mismatch: argument 2 requires (array or string) type, however, '10' is of int type. argument 3 requires (array or string) type, however, '20' is of int type. argument 5 requires (array or string) type, however, '30' is of int type.; line 1 pos 7 +cannot resolve 'concat_ws(',', 10, 20, NULL, 30)' due to data type mismatch: argument 2 requires (array or string) type, however, '10' is of int type. argument 3 requires (array or string) type, however, '20' is of int type. argument 4 requires (array or string) type, however, 'NULL' is of null type. argument 5 requires (array or string) type, however, '30' is of int type.; line 1 pos 7 -- !query @@ -139,7 +139,7 @@ select concat_ws('',10,20,null,30) struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve 'concat_ws('', 10, 20, NULL, 30)' due to data type mismatch: argument 2 requires (array or string) type, however, '10' is of int type. argument 3 requires (array or string) type, however, '20' is of int type. argument 5 requires (array or string) type, however, '30' is of int type.; line 1 pos 7 +cannot resolve 'concat_ws('', 10, 20, NULL, 30)' due to data type mismatch: argument 2 requires (array or string) type, however, '10' is of int type. argument 3 requires (array or string) type, however, '20' is of int type. argument 4 requires (array or string) type, however, 'NULL' is of null type. argument 5 requires (array or string) type, however, '30' is of int type.; line 1 pos 7 -- !query @@ -148,7 +148,7 @@ select concat_ws(NULL,10,20,null,30) is null struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve 'concat_ws(CAST(NULL AS STRING), 10, 20, NULL, 30)' due to data type mismatch: argument 2 requires (array or string) type, however, '10' is of int type. argument 3 requires (array or string) type, however, '20' is of int type. argument 5 requires (array or string) type, however, '30' is of int type.; line 1 pos 7 +cannot resolve 'concat_ws(CAST(NULL AS STRING), 10, 20, NULL, 30)' due to data type mismatch: argument 2 requires (array or string) type, however, '10' is of int type. argument 3 requires (array or string) type, however, '20' is of int type. argument 4 requires (array or string) type, however, 'NULL' is of null type. argument 5 requires (array or string) type, however, '30' is of int type.; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index 03a45a287d5d6..80e88d0566411 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 48 +-- Number of queries: 50 -- !query @@ -69,11 +69,19 @@ ab abcd ab NULL -- !query -select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a') +select left(null, -2) -- !query schema -struct +struct -- !query output -NULL NULL +NULL + + +-- !query +select left("abcd", -2), left("abcd", 0), left("abcd", 'a') +-- !query schema +struct +-- !query output + NULL -- !query @@ -85,11 +93,19 @@ cd abcd cd NULL -- !query -select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a') +select right(null, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select right("abcd", -2), right("abcd", 0), right("abcd", 'a') -- !query schema -struct +struct -- !query output -NULL NULL + NULL -- !query From 88cf86f56b6abcaa07bfd929ff0e29070e985766 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 24 Mar 2021 12:16:21 -0500 Subject: [PATCH 21/24] [SPARK-34797][ML] Refactor Logistic Aggregator - support virtual centering ### What changes were proposed in this pull request? 1, add `BinaryLogisticBlockAggregator` and `MultinomialLogisticBlockAggregator` and related testsuites; 2, impl `virtual centering` in standardization; 3, remove old `LogisticAggregator` ### Why are the changes needed? previous [pr](https://github.com/apache/spark/pull/31693) and related works is too large, we need to split it into 3 prs: 1, this one, impl new agg supporting `virtual centering`, remove old `LogisticAggregator`; 2, adopt new blor-agg in lor, add new test suite for small var features; 3, adopt new mlor-agg in lor, add new test suite for small var features, remove `BlockLogisticAggregator`; ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? added testsuites Closes #31889 from zhengruifeng/blor_mlor_agg. Authored-by: Ruifeng Zheng Signed-off-by: Sean Owen --- .../BinaryLogisticBlockAggregator.scala | 170 ++++++++ .../optim/aggregator/LogisticAggregator.scala | 346 +--------------- .../MultinomialLogisticBlockAggregator.scala | 212 ++++++++++ .../LogisticRegressionSuite.scala | 27 -- .../BinaryLogisticBlockAggregatorSuite.scala | 303 ++++++++++++++ .../aggregator/LogisticAggregatorSuite.scala | 333 --------------- ...tinomialLogisticBlockAggregatorSuite.scala | 387 ++++++++++++++++++ 7 files changed, 1073 insertions(+), 705 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregator.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregatorSuite.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregatorSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregator.scala new file mode 100644 index 0000000000000..091c885ca01f3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregator.scala @@ -0,0 +1,170 @@ +/* + * 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.ml.optim.aggregator + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.InstanceBlock +import org.apache.spark.ml.impl.Utils +import org.apache.spark.ml.linalg._ + +/** + * BinaryLogisticBlockAggregator computes the gradient and loss used in binary logistic + * classification for blocks in sparse or dense matrix in an online fashion. + * + * Two BinaryLogisticBlockAggregator can be merged together to have a summary of loss and + * gradient of the corresponding joint dataset. + * + * NOTE: The feature values are expected to already have be scaled (multiplied by bcInverseStd, + * but NOT centered) before computation. + * + * @param bcCoefficients The coefficients corresponding to the features. + * @param fitIntercept Whether to fit an intercept term. + * @param fitWithMean Whether to center the data with mean before training, in a virtual way. + * If true, we MUST adjust the intercept of both initial coefficients and + * final solution in the caller. + */ +private[ml] class BinaryLogisticBlockAggregator( + bcInverseStd: Broadcast[Array[Double]], + bcScaledMean: Broadcast[Array[Double]], + fitIntercept: Boolean, + fitWithMean: Boolean)(bcCoefficients: Broadcast[Vector]) + extends DifferentiableLossAggregator[InstanceBlock, BinaryLogisticBlockAggregator] + with Logging { + + if (fitWithMean) { + require(fitIntercept, s"for training without intercept, should not center the vectors") + require(bcScaledMean != null && bcScaledMean.value.length == bcInverseStd.value.length, + "scaled means is required when center the vectors") + } + + private val numFeatures = bcInverseStd.value.length + protected override val dim: Int = bcCoefficients.value.size + + @transient private lazy val coefficientsArray = bcCoefficients.value match { + case DenseVector(values) => values + case _ => throw new IllegalArgumentException(s"coefficients only supports dense vector but " + + s"got type ${bcCoefficients.value.getClass}.)") + } + + @transient private lazy val linear = if (fitIntercept) { + new DenseVector(coefficientsArray.take(numFeatures)) + } else { + new DenseVector(coefficientsArray) + } + + // pre-computed margin of an empty vector. + // with this variable as an offset, for a sparse vector, we only need to + // deal with non-zero values in prediction. + private val marginOffset = if (fitWithMean) { + coefficientsArray.last - + BLAS.getBLAS(numFeatures).ddot(numFeatures, coefficientsArray, 1, bcScaledMean.value, 1) + } else { + Double.NaN + } + + /** + * Add a new training instance block to this BinaryLogisticBlockAggregator, and update the loss + * and gradient of the objective function. + * + * @param block The instance block of data point to be added. + * @return This BinaryLogisticBlockAggregator object. + */ + def add(block: InstanceBlock): this.type = { + require(block.matrix.isTransposed) + require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + + s"instance. Expecting $numFeatures but got ${block.numFeatures}.") + require(block.weightIter.forall(_ >= 0), + s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") + + if (block.weightIter.forall(_ == 0)) return this + val size = block.size + + // vec/arr here represents margins + val vec = new DenseVector(Array.ofDim[Double](size)) + val arr = vec.values + if (fitIntercept) { + val offset = if (fitWithMean) marginOffset else coefficientsArray.last + java.util.Arrays.fill(arr, offset) + } + BLAS.gemv(1.0, block.matrix, linear, 1.0, vec) + + // in-place convert margins to multiplier + // then, vec/arr represents multiplier + var localLossSum = 0.0 + var localWeightSum = 0.0 + var multiplierSum = 0.0 + var i = 0 + while (i < size) { + val weight = block.getWeight(i) + localWeightSum += weight + if (weight > 0) { + val label = block.getLabel(i) + val margin = arr(i) + if (label > 0) { + // The following is equivalent to log(1 + exp(-margin)) but more numerically stable. + localLossSum += weight * Utils.log1pExp(-margin) + } else { + localLossSum += weight * (Utils.log1pExp(-margin) + margin) + } + val multiplier = weight * (1.0 / (1.0 + math.exp(-margin)) - label) + arr(i) = multiplier + multiplierSum += multiplier + } else { arr(i) = 0.0 } + i += 1 + } + lossSum += localLossSum + weightSum += localWeightSum + + // predictions are all correct, no gradient signal + if (arr.forall(_ == 0)) return this + + // update the linear part of gradientSumArray + block.matrix match { + case dm: DenseMatrix => + BLAS.nativeBLAS.dgemv("N", dm.numCols, dm.numRows, 1.0, dm.values, dm.numCols, + vec.values, 1, 1.0, gradientSumArray, 1) + + case sm: SparseMatrix if fitIntercept => + val linearGradSumVec = new DenseVector(Array.ofDim[Double](numFeatures)) + BLAS.gemv(1.0, sm.transpose, vec, 0.0, linearGradSumVec) + BLAS.getBLAS(numFeatures).daxpy(numFeatures, 1.0, linearGradSumVec.values, 1, + gradientSumArray, 1) + + case sm: SparseMatrix if !fitIntercept => + val gradSumVec = new DenseVector(gradientSumArray) + BLAS.gemv(1.0, sm.transpose, vec, 1.0, gradSumVec) + + case m => + throw new IllegalArgumentException(s"Unknown matrix type ${m.getClass}.") + } + + if (fitWithMean) { + // above update of the linear part of gradientSumArray does NOT take the centering + // into account, here we need to adjust this part. + BLAS.getBLAS(numFeatures).daxpy(numFeatures, -multiplierSum, bcScaledMean.value, 1, + gradientSumArray, 1) + } + + if (fitIntercept) { + // update the intercept part of gradientSumArray + gradientSumArray(numFeatures) += multiplierSum + } + + this + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala index 5a516940b9788..a9199491129e6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala @@ -18,354 +18,10 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.feature.InstanceBlock import org.apache.spark.ml.impl.Utils import org.apache.spark.ml.linalg._ -/** - * LogisticAggregator computes the gradient and loss for binary or multinomial logistic (softmax) - * loss function, as used in classification for instances in sparse or dense vector in an online - * fashion. - * - * Two LogisticAggregators can be merged together to have a summary of loss and gradient of - * the corresponding joint dataset. - * - * For improving the convergence rate during the optimization process and also to prevent against - * features with very large variances exerting an overly large influence during model training, - * packages like R's GLMNET perform the scaling to unit variance and remove the mean in order to - * reduce the condition number. The model is then trained in this scaled space, but returns the - * coefficients in the original scale. See page 9 in - * http://cran.r-project.org/web/packages/glmnet/glmnet.pdf - * - * However, we don't want to apply the [[org.apache.spark.ml.feature.StandardScaler]] on the - * training dataset, and then cache the standardized dataset since it will create a lot of overhead. - * As a result, we perform the scaling implicitly when we compute the objective function (though - * we do not subtract the mean). - * - * Note that there is a difference between multinomial (softmax) and binary loss. The binary case - * uses one outcome class as a "pivot" and regresses the other class against the pivot. In the - * multinomial case, the softmax loss function is used to model each class probability - * independently. Using softmax loss produces `K` sets of coefficients, while using a pivot class - * produces `K - 1` sets of coefficients (a single coefficient vector in the binary case). In the - * binary case, we can say that the coefficients are shared between the positive and negative - * classes. When regularization is applied, multinomial (softmax) loss will produce a result - * different from binary loss since the positive and negative don't share the coefficients while the - * binary regression shares the coefficients between positive and negative. - * - * The following is a mathematical derivation for the multinomial (softmax) loss. - * - * The probability of the multinomial outcome $y$ taking on any of the K possible outcomes is: - * - *
- * $$ - * P(y_i=0|\vec{x}_i, \beta) = \frac{e^{\vec{x}_i^T \vec{\beta}_0}}{\sum_{k=0}^{K-1} - * e^{\vec{x}_i^T \vec{\beta}_k}} \\ - * P(y_i=1|\vec{x}_i, \beta) = \frac{e^{\vec{x}_i^T \vec{\beta}_1}}{\sum_{k=0}^{K-1} - * e^{\vec{x}_i^T \vec{\beta}_k}}\\ - * P(y_i=K-1|\vec{x}_i, \beta) = \frac{e^{\vec{x}_i^T \vec{\beta}_{K-1}}\,}{\sum_{k=0}^{K-1} - * e^{\vec{x}_i^T \vec{\beta}_k}} - * $$ - *
- * - * The model coefficients $\beta = (\beta_0, \beta_1, \beta_2, ..., \beta_{K-1})$ become a matrix - * which has dimension of $K \times (N+1)$ if the intercepts are added. If the intercepts are not - * added, the dimension will be $K \times N$. - * - * Note that the coefficients in the model above lack identifiability. That is, any constant scalar - * can be added to all of the coefficients and the probabilities remain the same. - * - *
- * $$ - * \begin{align} - * \frac{e^{\vec{x}_i^T \left(\vec{\beta}_0 + \vec{c}\right)}}{\sum_{k=0}^{K-1} - * e^{\vec{x}_i^T \left(\vec{\beta}_k + \vec{c}\right)}} - * = \frac{e^{\vec{x}_i^T \vec{\beta}_0}e^{\vec{x}_i^T \vec{c}}\,}{e^{\vec{x}_i^T \vec{c}} - * \sum_{k=0}^{K-1} e^{\vec{x}_i^T \vec{\beta}_k}} - * = \frac{e^{\vec{x}_i^T \vec{\beta}_0}}{\sum_{k=0}^{K-1} e^{\vec{x}_i^T \vec{\beta}_k}} - * \end{align} - * $$ - *
- * - * However, when regularization is added to the loss function, the coefficients are indeed - * identifiable because there is only one set of coefficients which minimizes the regularization - * term. When no regularization is applied, we choose the coefficients with the minimum L2 - * penalty for consistency and reproducibility. For further discussion see: - * - * Friedman, et al. "Regularization Paths for Generalized Linear Models via Coordinate Descent" - * - * The loss of objective function for a single instance of data (we do not include the - * regularization term here for simplicity) can be written as - * - *
- * $$ - * \begin{align} - * \ell\left(\beta, x_i\right) &= -log{P\left(y_i \middle| \vec{x}_i, \beta\right)} \\ - * &= log\left(\sum_{k=0}^{K-1}e^{\vec{x}_i^T \vec{\beta}_k}\right) - \vec{x}_i^T \vec{\beta}_y\\ - * &= log\left(\sum_{k=0}^{K-1} e^{margins_k}\right) - margins_y - * \end{align} - * $$ - *
- * - * where ${margins}_k = \vec{x}_i^T \vec{\beta}_k$. - * - * For optimization, we have to calculate the first derivative of the loss function, and a simple - * calculation shows that - * - *
- * $$ - * \begin{align} - * \frac{\partial \ell(\beta, \vec{x}_i, w_i)}{\partial \beta_{j, k}} - * &= x_{i,j} \cdot w_i \cdot \left(\frac{e^{\vec{x}_i \cdot \vec{\beta}_k}}{\sum_{k'=0}^{K-1} - * e^{\vec{x}_i \cdot \vec{\beta}_{k'}}\,} - I_{y=k}\right) \\ - * &= x_{i, j} \cdot w_i \cdot multiplier_k - * \end{align} - * $$ - *
- * - * where $w_i$ is the sample weight, $I_{y=k}$ is an indicator function - * - *
- * $$ - * I_{y=k} = \begin{cases} - * 1 & y = k \\ - * 0 & else - * \end{cases} - * $$ - *
- * - * and - * - *
- * $$ - * multiplier_k = \left(\frac{e^{\vec{x}_i \cdot \vec{\beta}_k}}{\sum_{k=0}^{K-1} - * e^{\vec{x}_i \cdot \vec{\beta}_k}} - I_{y=k}\right) - * $$ - *
- * - * If any of margins is larger than 709.78, the numerical computation of multiplier and loss - * function will suffer from arithmetic overflow. This issue occurs when there are outliers in - * data which are far away from the hyperplane, and this will cause the failing of training once - * infinity is introduced. Note that this is only a concern when max(margins) > 0. - * - * Fortunately, when max(margins) = maxMargin > 0, the loss function and the multiplier can - * easily be rewritten into the following equivalent numerically stable formula. - * - *
- * $$ - * \ell\left(\beta, x\right) = log\left(\sum_{k=0}^{K-1} e^{margins_k - maxMargin}\right) - - * margins_{y} + maxMargin - * $$ - *
- * - * Note that each term, $(margins_k - maxMargin)$ in the exponential is no greater than zero; as a - * result, overflow will not happen with this formula. - * - * For $multiplier$, a similar trick can be applied as the following, - * - *
- * $$ - * multiplier_k = \left(\frac{e^{\vec{x}_i \cdot \vec{\beta}_k - maxMargin}}{\sum_{k'=0}^{K-1} - * e^{\vec{x}_i \cdot \vec{\beta}_{k'} - maxMargin}} - I_{y=k}\right) - * $$ - *
- * - * - * @param bcCoefficients The broadcast coefficients corresponding to the features. - * @param bcFeaturesStd The broadcast standard deviation values of the features. - * @param numClasses the number of possible outcomes for k classes classification problem in - * Multinomial Logistic Regression. - * @param fitIntercept Whether to fit an intercept term. - * @param multinomial Whether to use multinomial (softmax) or binary loss - * @note In order to avoid unnecessary computation during calculation of the gradient updates - * we lay out the coefficients in column major order during training. This allows us to - * perform feature standardization once, while still retaining sequential memory access - * for speed. We convert back to row major order when we create the model, - * since this form is optimal for the matrix operations used for prediction. - */ -private[ml] class LogisticAggregator( - bcFeaturesStd: Broadcast[Array[Double]], - numClasses: Int, - fitIntercept: Boolean, - multinomial: Boolean)(bcCoefficients: Broadcast[Vector]) - extends DifferentiableLossAggregator[Instance, LogisticAggregator] with Logging { - - private val numFeatures = bcFeaturesStd.value.length - private val numFeaturesPlusIntercept = if (fitIntercept) numFeatures + 1 else numFeatures - private val coefficientSize = bcCoefficients.value.size - protected override val dim: Int = coefficientSize - if (multinomial) { - require(numClasses == coefficientSize / numFeaturesPlusIntercept, s"The number of " + - s"coefficients should be ${numClasses * numFeaturesPlusIntercept} but was $coefficientSize") - } else { - require(coefficientSize == numFeaturesPlusIntercept, s"Expected $numFeaturesPlusIntercept " + - s"coefficients but got $coefficientSize") - require(numClasses == 1 || numClasses == 2, s"Binary logistic aggregator requires numClasses " + - s"in {1, 2} but found $numClasses.") - } - - @transient private lazy val coefficientsArray = bcCoefficients.value match { - case DenseVector(values) => values - case _ => throw new IllegalArgumentException(s"coefficients only supports dense vector but " + - s"got type ${bcCoefficients.value.getClass}.)") - } - - if (multinomial && numClasses <= 2) { - logInfo(s"Multinomial logistic regression for binary classification yields separate " + - s"coefficients for positive and negative classes. When no regularization is applied, the" + - s"result will be effectively the same as binary logistic regression. When regularization" + - s"is applied, multinomial loss will produce a result different from binary loss.") - } - - /** Update gradient and loss using binary loss function. */ - private def binaryUpdateInPlace(features: Vector, weight: Double, label: Double): Unit = { - - val localFeaturesStd = bcFeaturesStd.value - val localCoefficients = coefficientsArray - val localGradientArray = gradientSumArray - val margin = - { - var sum = 0.0 - features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - sum += localCoefficients(index) * value / localFeaturesStd(index) - } - } - if (fitIntercept) sum += localCoefficients(numFeaturesPlusIntercept - 1) - sum - } - - val multiplier = weight * (1.0 / (1.0 + math.exp(margin)) - label) - - features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - localGradientArray(index) += multiplier * value / localFeaturesStd(index) - } - } - - if (fitIntercept) { - localGradientArray(numFeaturesPlusIntercept - 1) += multiplier - } - - if (label > 0) { - // The following is equivalent to log(1 + exp(margin)) but more numerically stable. - lossSum += weight * Utils.log1pExp(margin) - } else { - lossSum += weight * (Utils.log1pExp(margin) - margin) - } - } - - /** Update gradient and loss using multinomial (softmax) loss function. */ - private def multinomialUpdateInPlace(features: Vector, weight: Double, label: Double): Unit = { - // TODO: use level 2 BLAS operations - /* - Note: this can still be used when numClasses = 2 for binary - logistic regression without pivoting. - */ - val localFeaturesStd = bcFeaturesStd.value - val localCoefficients = coefficientsArray - val localGradientArray = gradientSumArray - - // marginOfLabel is margins(label) in the formula - var marginOfLabel = 0.0 - var maxMargin = Double.NegativeInfinity - - val margins = new Array[Double](numClasses) - features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - val stdValue = value / localFeaturesStd(index) - var j = 0 - while (j < numClasses) { - margins(j) += localCoefficients(index * numClasses + j) * stdValue - j += 1 - } - } - } - var i = 0 - while (i < numClasses) { - if (fitIntercept) { - margins(i) += localCoefficients(numClasses * numFeatures + i) - } - if (i == label.toInt) marginOfLabel = margins(i) - if (margins(i) > maxMargin) { - maxMargin = margins(i) - } - i += 1 - } - - /** - * When maxMargin is greater than 0, the original formula could cause overflow. - * We address this by subtracting maxMargin from all the margins, so it's guaranteed - * that all of the new margins will be smaller than zero to prevent arithmetic overflow. - */ - val multipliers = new Array[Double](numClasses) - val sum = { - var temp = 0.0 - var i = 0 - while (i < numClasses) { - if (maxMargin > 0) margins(i) -= maxMargin - val exp = math.exp(margins(i)) - temp += exp - multipliers(i) = exp - i += 1 - } - temp - } - - margins.indices.foreach { i => - multipliers(i) = multipliers(i) / sum - (if (label == i) 1.0 else 0.0) - } - features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - val stdValue = value / localFeaturesStd(index) - var j = 0 - while (j < numClasses) { - localGradientArray(index * numClasses + j) += weight * multipliers(j) * stdValue - j += 1 - } - } - } - if (fitIntercept) { - var i = 0 - while (i < numClasses) { - localGradientArray(numFeatures * numClasses + i) += weight * multipliers(i) - i += 1 - } - } - - val loss = if (maxMargin > 0) { - math.log(sum) - marginOfLabel + maxMargin - } else { - math.log(sum) - marginOfLabel - } - lossSum += weight * loss - } - - /** - * Add a new training instance to this LogisticAggregator, and update the loss and gradient - * of the objective function. - * - * @param instance The instance of data point to be added. - * @return This LogisticAggregator object. - */ - def add(instance: Instance): this.type = { - instance match { case Instance(label, weight, features) => - require(numFeatures == features.size, s"Dimensions mismatch when adding new instance." + - s" Expecting $numFeatures but got ${features.size}.") - require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") - - if (weight == 0.0) return this - - if (multinomial) { - multinomialUpdateInPlace(features, weight, label) - } else { - binaryUpdateInPlace(features, weight, label) - } - weightSum += weight - this - } - } -} - - /** * BlockLogisticAggregator computes the gradient and loss used in Logistic classification * for blocks in sparse or dense matrix in an online fashion. diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregator.scala new file mode 100644 index 0000000000000..de6444084379e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregator.scala @@ -0,0 +1,212 @@ +/* + * 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.ml.optim.aggregator + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.InstanceBlock +import org.apache.spark.ml.linalg._ + +/** + * MultinomialLogisticBlockAggregator computes the gradient and loss used in multinomial logistic + * classification for blocks in sparse or dense matrix in an online fashion. + * + * Two MultinomialLogisticBlockAggregator can be merged together to have a summary of loss and + * gradient of the corresponding joint dataset. + * + * NOTE: The feature values are expected to already have be scaled (multiplied by bcInverseStd, + * but NOT centered) before computation. + * + * @param bcCoefficients The coefficients corresponding to the features. + * @param fitIntercept Whether to fit an intercept term. + * @param fitWithMean Whether to center the data with mean before training, in a virtual way. + * If true, we MUST adjust the intercept of both initial coefficients and + * final solution in the caller. + * @note In order to avoid unnecessary computation during calculation of the gradient updates + * we lay out the coefficients in column major order during training. We convert back to row + * major order when we create the model, since this form is optimal for the matrix operations + * used for prediction. + */ +private[ml] class MultinomialLogisticBlockAggregator( + bcInverseStd: Broadcast[Array[Double]], + bcScaledMean: Broadcast[Array[Double]], + fitIntercept: Boolean, + fitWithMean: Boolean)(bcCoefficients: Broadcast[Vector]) + extends DifferentiableLossAggregator[InstanceBlock, MultinomialLogisticBlockAggregator] + with Logging { + + if (fitWithMean) { + require(fitIntercept, s"for training without intercept, should not center the vectors") + require(bcScaledMean != null && bcScaledMean.value.length == bcInverseStd.value.length, + "scaled means is required when center the vectors") + } + + private val numFeatures = bcInverseStd.value.length + protected override val dim: Int = bcCoefficients.value.size + private val numFeaturesPlusIntercept = if (fitIntercept) numFeatures + 1 else numFeatures + private val numClasses = dim / numFeaturesPlusIntercept + require(dim == numClasses * numFeaturesPlusIntercept) + + @transient private lazy val coefficientsArray = bcCoefficients.value match { + case DenseVector(values) => values + case _ => throw new IllegalArgumentException(s"coefficients only supports dense vector but " + + s"got type ${bcCoefficients.value.getClass}.)") + } + + @transient private lazy val linear = if (fitIntercept) { + new DenseMatrix(numClasses, numFeatures, coefficientsArray.take(numClasses * numFeatures)) + } else { + new DenseMatrix(numClasses, numFeatures, coefficientsArray) + } + + private lazy val intercept = if (fitIntercept) { + new DenseVector(coefficientsArray.takeRight(numClasses)) + } else { + null + } + + // pre-computed margin of an empty vector. + // with this variable as an offset, for a sparse vector, we only need to + // deal with non-zero values in prediction. + private val marginOffset = if (fitWithMean) { + val offset = intercept.copy + BLAS.gemv(-1.0, linear, Vectors.dense(bcScaledMean.value), 1.0, offset) + offset + } else { + null + } + + /** + * Add a new training instance block to this BinaryLogisticBlockAggregator, and update the loss + * and gradient of the objective function. + * + * @param block The instance block of data point to be added. + * @return This BinaryLogisticBlockAggregator object. + */ + def add(block: InstanceBlock): this.type = { + require(block.matrix.isTransposed) + require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + + s"instance. Expecting $numFeatures but got ${block.numFeatures}.") + require(block.weightIter.forall(_ >= 0), + s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") + + if (block.weightIter.forall(_ == 0)) return this + val size = block.size + + // mat/arr here represents margins, shape: S X C + val mat = DenseMatrix.zeros(size, numClasses) + val arr = mat.values + if (fitIntercept) { + val offset = if (fitWithMean) marginOffset else intercept + var j = 0 + while (j < numClasses) { + java.util.Arrays.fill(arr, j * size, (j + 1) * size, offset(j)) + j += 1 + } + } + BLAS.gemm(1.0, block.matrix, linear.transpose, 1.0, mat) + + // in-place convert margins to multipliers + // then, mat/arr represents multipliers + var localLossSum = 0.0 + var localWeightSum = 0.0 + var i = 0 + val tmp = Array.ofDim[Double](numClasses) + val multiplierSum = Array.ofDim[Double](numClasses) + while (i < size) { + val weight = block.getWeight(i) + localWeightSum += weight + if (weight > 0) { + val label = block.getLabel(i) + var maxMargin = Double.NegativeInfinity + var j = 0 + while (j < numClasses) { + tmp(j) = mat(i, j) + maxMargin = math.max(maxMargin, tmp(j)) + j += 1 + } + + // marginOfLabel is margins(label) in the formula + val marginOfLabel = tmp(label.toInt) + + var sum = 0.0 + j = 0 + while (j < numClasses) { + if (maxMargin > 0) tmp(j) -= maxMargin + val exp = math.exp(tmp(j)) + sum += exp + tmp(j) = exp + j += 1 + } + + j = 0 + while (j < numClasses) { + val multiplier = weight * (tmp(j) / sum - (if (label == j) 1.0 else 0.0)) + mat.update(i, j, multiplier) + multiplierSum(j) += multiplier + j += 1 + } + + if (maxMargin > 0) { + localLossSum += weight * (math.log(sum) - marginOfLabel + maxMargin) + } else { + localLossSum += weight * (math.log(sum) - marginOfLabel) + } + } else { + var j = 0; while (j < numClasses) { mat.update(i, j, 0.0); j += 1 } + } + i += 1 + } + lossSum += localLossSum + weightSum += localWeightSum + + // mat (multipliers): S X C, dense N + // mat.transpose (multipliers): C X S, dense T + // block.matrix (data): S X F, unknown type T + // gradSumMat (gradientSumArray): C X FPI (numFeaturesPlusIntercept), dense N + block.matrix match { + case dm: DenseMatrix => + // gradientSumArray[0 : F X C] += mat.T X dm + BLAS.nativeBLAS.dgemm("T", "T", numClasses, numFeatures, size, 1.0, + mat.values, size, dm.values, numFeatures, 1.0, gradientSumArray, numClasses) + + case sm: SparseMatrix => + // TODO: convert Coefficients to row major order to simplify BLAS operations? + // linearGradSumMat = sm.T X mat + // existing BLAS.gemm requires linearGradSumMat is NOT Transposed. + val linearGradSumMat = DenseMatrix.zeros(numFeatures, numClasses) + BLAS.gemm(1.0, sm.transpose, mat, 0.0, linearGradSumMat) + linearGradSumMat.foreachActive { (i, j, v) => gradientSumArray(i * numClasses + j) += v } + } + + if (fitWithMean) { + // above update of the linear part of gradientSumArray does NOT take the centering + // into account, here we need to adjust this part. + // following BLAS.dger operation equals to: gradientSumArray[0 : F X C] -= mat.T X _mm_, + // where _mm_ is a matrix of size S X F with each row equals to array ScaledMean. + BLAS.nativeBLAS.dger(numClasses, numFeatures, -1.0, multiplierSum, 1, + bcScaledMean.value, 1, gradientSumArray, numClasses) + } + + if (fitIntercept) { + BLAS.getBLAS(numClasses).daxpy(numClasses, 1.0, multiplierSum, 0, 1, + gradientSumArray, numClasses * numFeatures, 1) + } + + this + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index d2814b420e017..3e2344bbcb6a2 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -28,7 +28,6 @@ import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.classification.LogisticRegressionSuite._ import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{DenseMatrix, Matrices, Matrix, SparseMatrix, Vector, Vectors} -import org.apache.spark.ml.optim.aggregator.LogisticAggregator import org.apache.spark.ml.param.{ParamMap, ParamsSuite} import org.apache.spark.ml.stat.MultiClassSummarizer import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} @@ -632,32 +631,6 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(blrModel.intercept !== 0.0) } - test("sparse coefficients in LogisticAggregator") { - val bcCoefficientsBinary = spark.sparkContext.broadcast(Vectors.sparse(2, Array(0), Array(1.0))) - val bcFeaturesStd = spark.sparkContext.broadcast(Array(1.0)) - val binaryAgg = new LogisticAggregator(bcFeaturesStd, 2, - fitIntercept = true, multinomial = false)(bcCoefficientsBinary) - val thrownBinary = withClue("binary logistic aggregator cannot handle sparse coefficients") { - intercept[IllegalArgumentException] { - binaryAgg.add(Instance(1.0, 1.0, Vectors.dense(1.0))) - } - } - assert(thrownBinary.getMessage.contains("coefficients only supports dense")) - - val bcCoefficientsMulti = spark.sparkContext.broadcast(Vectors.sparse(6, Array(0), Array(1.0))) - val multinomialAgg = new LogisticAggregator(bcFeaturesStd, 3, - fitIntercept = true, multinomial = true)(bcCoefficientsMulti) - val thrown = withClue("multinomial logistic aggregator cannot handle sparse coefficients") { - intercept[IllegalArgumentException] { - multinomialAgg.add(Instance(1.0, 1.0, Vectors.dense(1.0))) - } - } - assert(thrown.getMessage.contains("coefficients only supports dense")) - bcCoefficientsBinary.destroy() - bcFeaturesStd.destroy() - bcCoefficientsMulti.destroy() - } - test("overflow prediction for multiclass") { val model = new LogisticRegressionModel("mLogReg", Matrices.dense(3, 2, Array(0.0, 0.0, 0.0, 1.0, 2.0, 3.0)), diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregatorSuite.scala new file mode 100644 index 0000000000000..f5ae22d228cdd --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregatorSuite.scala @@ -0,0 +1,303 @@ +/* + * 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.ml.optim.aggregator + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.stat.Summarizer +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class BinaryLogisticBlockAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { + + @transient var instances: Array[Instance] = _ + @transient var instancesConstantFeature: Array[Instance] = _ + @transient var instancesConstantFeatureFiltered: Array[Instance] = _ + @transient var scaledInstances: Array[Instance] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + instances = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), + Instance(0.0, 0.3, Vectors.dense(4.0, 0.5)) + ) + instancesConstantFeature = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), + Instance(0.0, 0.3, Vectors.dense(1.0, 0.5)) + ) + instancesConstantFeatureFiltered = Array( + Instance(0.0, 0.1, Vectors.dense(2.0)), + Instance(1.0, 0.5, Vectors.dense(1.0)), + Instance(0.0, 0.3, Vectors.dense(0.5)) + ) + scaledInstances = standardize(instances) + } + + /** Get summary statistics for some data and create a new BinaryLogisticBlockAggregator. */ + private def getNewAggregator( + instances: Array[Instance], + coefficients: Vector, + fitIntercept: Boolean, + fitWithMean: Boolean): BinaryLogisticBlockAggregator = { + val (featuresSummarizer, _) = + Summarizer.getClassificationSummarizers(sc.parallelize(instances)) + val featuresStd = featuresSummarizer.std.toArray + val featuresMean = featuresSummarizer.mean.toArray + val inverseStd = featuresStd.map(std => if (std != 0) 1.0 / std else 0.0) + val scaledMean = inverseStd.zip(featuresMean).map(t => t._1 * t._2) + val bcInverseStd = sc.broadcast(inverseStd) + val bcScaledMean = sc.broadcast(scaledMean) + val bcCoefficients = sc.broadcast(coefficients) + new BinaryLogisticBlockAggregator(bcInverseStd, + bcScaledMean, fitIntercept, fitWithMean)(bcCoefficients) + } + + test("sparse coefficients") { + val bcInverseStd = sc.broadcast(Array(1.0)) + val bcScaledMean = sc.broadcast(Array(2.0)) + val bcCoefficients = sc.broadcast(Vectors.sparse(2, Array(0), Array(1.0))) + val binaryAgg = new BinaryLogisticBlockAggregator(bcInverseStd, bcScaledMean, + fitIntercept = true, fitWithMean = false)(bcCoefficients) + val block = InstanceBlock.fromInstances(Seq(Instance(1.0, 1.0, Vectors.dense(1.0)))) + val thrownBinary = withClue("aggregator cannot handle sparse coefficients") { + intercept[IllegalArgumentException] { + binaryAgg.add(block) + } + } + assert(thrownBinary.getMessage.contains("coefficients only supports dense")) + } + + test("aggregator add method input size") { + val coefArray = Array(1.0, 2.0) + val interceptValue = 4.0 + val agg = getNewAggregator(instances, Vectors.dense(coefArray :+ interceptValue), + fitIntercept = true, fitWithMean = true) + val block = InstanceBlock.fromInstances(Seq(Instance(1.0, 1.0, Vectors.dense(2.0)))) + withClue("BinaryLogisticBlockAggregator features dimension must match coefficients dimension") { + intercept[IllegalArgumentException] { + agg.add(block) + } + } + } + + test("negative weight") { + val coefArray = Array(1.0, 2.0) + val interceptValue = 4.0 + val agg = getNewAggregator(instances, Vectors.dense(coefArray :+ interceptValue), + fitIntercept = true, fitWithMean = true) + val block = InstanceBlock.fromInstances(Seq(Instance(1.0, -1.0, Vectors.dense(2.0, 1.0)))) + withClue("BinaryLogisticBlockAggregator does not support negative instance weights") { + intercept[IllegalArgumentException] { + agg.add(block) + } + } + } + + test("check sizes") { + val rng = new scala.util.Random + val numFeatures = instances.head.features.size + val coefWithIntercept = Vectors.dense(Array.fill(numFeatures + 1)(rng.nextDouble)) + val coefWithoutIntercept = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble)) + val block = InstanceBlock.fromInstances(instances) + + val aggIntercept = getNewAggregator(instances, coefWithIntercept, + fitIntercept = true, fitWithMean = false) + aggIntercept.add(block) + assert(aggIntercept.gradient.size === numFeatures + 1) + + val aggNoIntercept = getNewAggregator(instances, coefWithoutIntercept, + fitIntercept = false, fitWithMean = false) + aggNoIntercept.add(block) + assert(aggNoIntercept.gradient.size === numFeatures) + } + + test("check correctness: fitIntercept = false") { + val coefVec = Vectors.dense(1.0, 2.0) + val numFeatures = instances.head.features.size + val (featuresSummarizer, _) = + Summarizer.getClassificationSummarizers(sc.parallelize(instances)) + val featuresStd = featuresSummarizer.std + val stdCoefVec = Vectors.dense(Array.tabulate(coefVec.size)(i => coefVec(i) / featuresStd(i))) + val weightSum = instances.map(_.weight).sum + + // compute the loss and the gradients + var lossSum = 0.0 + val gradientCoef = Array.ofDim[Double](numFeatures) + instances.foreach { case Instance(l, w, f) => + val margin = BLAS.dot(stdCoefVec, f) + val prob = 1.0 / (1.0 + math.exp(-margin)) + lossSum += -w * l * math.log(prob) - w * (1.0 - l) * math.log1p(-prob) + gradientCoef.indices.foreach { i => + gradientCoef(i) += w * (prob - l) * f(i) / featuresStd(i) + } + } + val loss = lossSum / weightSum + val gradient = Vectors.dense(gradientCoef.map(_ / weightSum)) + + Seq(1, 2, 4).foreach { blockSize => + val blocks1 = scaledInstances + .grouped(blockSize) + .map(seq => InstanceBlock.fromInstances(seq)) + .toArray + val blocks2 = blocks1.map { block => + new InstanceBlock(block.labels, block.weights, block.matrix.toSparseRowMajor) + } + + Seq(blocks1, blocks2).foreach { blocks => + val agg = getNewAggregator(instances, coefVec, + fitIntercept = false, fitWithMean = false) + blocks.foreach(agg.add) + assert(agg.loss ~== loss relTol 1e-9) + assert(agg.gradient ~== gradient relTol 1e-9) + } + } + } + + test("check correctness: fitIntercept = true, fitWithMean = false") { + val coefVec = Vectors.dense(1.0, 2.0) + val interceptValue = 1.0 + val numFeatures = instances.head.features.size + val (featuresSummarizer, _) = + Summarizer.getClassificationSummarizers(sc.parallelize(instances)) + val featuresStd = featuresSummarizer.std + val stdCoefVec = Vectors.dense(Array.tabulate(coefVec.size)(i => coefVec(i) / featuresStd(i))) + val weightSum = instances.map(_.weight).sum + + // compute the loss and the gradients + var lossSum = 0.0 + val gradientCoef = Array.ofDim[Double](numFeatures) + var gradientIntercept = 0.0 + instances.foreach { case Instance(l, w, f) => + val margin = BLAS.dot(stdCoefVec, f) + interceptValue + val prob = 1.0 / (1.0 + math.exp(-margin)) + lossSum += -w * l * math.log(prob) - w * (1.0 - l) * math.log1p(-prob) + gradientCoef.indices.foreach { i => + gradientCoef(i) += w * (prob - l) * f(i) / featuresStd(i) + } + gradientIntercept += w * (prob - l) + } + val loss = lossSum / weightSum + val gradient = Vectors.dense((gradientCoef :+ gradientIntercept).map(_ / weightSum)) + + Seq(1, 2, 4).foreach { blockSize => + val blocks1 = scaledInstances + .grouped(blockSize) + .map(seq => InstanceBlock.fromInstances(seq)) + .toArray + val blocks2 = blocks1.map { block => + new InstanceBlock(block.labels, block.weights, block.matrix.toSparseRowMajor) + } + + Seq(blocks1, blocks2).foreach { blocks => + val agg = getNewAggregator(instances, Vectors.dense(coefVec.toArray :+ interceptValue), + fitIntercept = true, fitWithMean = false) + blocks.foreach(agg.add) + assert(agg.loss ~== loss relTol 1e-9) + assert(agg.gradient ~== gradient relTol 1e-9) + } + } + } + + test("check correctness: fitIntercept = true, fitWithMean = true") { + val coefVec = Vectors.dense(1.0, 2.0) + val interceptValue = 1.0 + val numFeatures = instances.head.features.size + val (featuresSummarizer, _) = + Summarizer.getClassificationSummarizers(sc.parallelize(instances)) + val featuresStd = featuresSummarizer.std + val featuresMean = featuresSummarizer.mean + val stdCoefVec = Vectors.dense(Array.tabulate(coefVec.size)(i => coefVec(i) / featuresStd(i))) + val weightSum = instances.map(_.weight).sum + + // compute the loss and the gradients + var lossSum = 0.0 + val gradientCoef = Array.ofDim[Double](numFeatures) + var gradientIntercept = 0.0 + instances.foreach { case Instance(l, w, f) => + val centered = f.toDense.copy + BLAS.axpy(-1.0, featuresMean, centered) + val margin = BLAS.dot(stdCoefVec, centered) + interceptValue + val prob = 1.0 / (1.0 + math.exp(-margin)) + lossSum += -w * l * math.log(prob) - w * (1.0 - l) * math.log1p(-prob) + gradientCoef.indices.foreach { i => + gradientCoef(i) += w * (prob - l) * (f(i) - featuresMean(i)) / featuresStd(i) + } + gradientIntercept += w * (prob - l) + } + val loss = lossSum / weightSum + val gradient = Vectors.dense((gradientCoef :+ gradientIntercept).map(_ / weightSum)) + + Seq(1, 2, 4).foreach { blockSize => + val blocks1 = scaledInstances + .grouped(blockSize) + .map(seq => InstanceBlock.fromInstances(seq)) + .toArray + val blocks2 = blocks1.map { block => + new InstanceBlock(block.labels, block.weights, block.matrix.toSparseRowMajor) + } + + Seq(blocks1, blocks2).foreach { blocks => + val agg = getNewAggregator(instances, Vectors.dense(coefVec.toArray :+ interceptValue), + fitIntercept = true, fitWithMean = true) + blocks.foreach(agg.add) + assert(agg.loss ~== loss relTol 1e-9) + assert(agg.gradient ~== gradient relTol 1e-9) + } + } + } + + test("check with zero standard deviation") { + val coefArray = Array(1.0, 2.0) + val coefArrayFiltered = Array(2.0) + val interceptValue = 1.0 + + Seq((false, false), (true, false), (true, true)).foreach { case (fitIntercept, fitWithMean) => + val coefVec = if (fitIntercept) { + Vectors.dense(coefArray :+ interceptValue) + } else { + Vectors.dense(coefArray) + } + val aggConstantFeature = getNewAggregator(instancesConstantFeature, + coefVec, fitIntercept = fitIntercept, fitWithMean = fitWithMean) + aggConstantFeature + .add(InstanceBlock.fromInstances(standardize(instancesConstantFeature))) + val grad = aggConstantFeature.gradient + + val coefVecFiltered = if (fitIntercept) { + Vectors.dense(coefArrayFiltered :+ interceptValue) + } else { + Vectors.dense(coefArrayFiltered) + } + val aggConstantFeatureFiltered = getNewAggregator(instancesConstantFeatureFiltered, + coefVecFiltered, fitIntercept = fitIntercept, fitWithMean = fitWithMean) + aggConstantFeatureFiltered + .add(InstanceBlock.fromInstances(standardize(instancesConstantFeatureFiltered))) + val gradFiltered = aggConstantFeatureFiltered.gradient + + // constant features should not affect gradient + assert(aggConstantFeature.loss ~== aggConstantFeatureFiltered.loss relTol 1e-9) + assert(grad(0) === 0) + assert(grad(1) ~== gradFiltered(0) relTol 1e-9) + if (fitIntercept) { + assert(grad.toArray.last ~== gradFiltered.toArray.last relTol 1e-9) + } + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala deleted file mode 100644 index e3e39c691b8a3..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala +++ /dev/null @@ -1,333 +0,0 @@ -/* - * 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.ml.optim.aggregator - -import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.{Instance, InstanceBlock} -import org.apache.spark.ml.linalg.{BLAS, Matrices, Vector, Vectors} -import org.apache.spark.ml.stat.Summarizer -import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.mllib.util.MLlibTestSparkContext - -class LogisticAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { - - @transient var instances: Array[Instance] = _ - @transient var instancesConstantFeature: Array[Instance] = _ - @transient var instancesConstantFeatureFiltered: Array[Instance] = _ - @transient var standardizedInstances: Array[Instance] = _ - - override def beforeAll(): Unit = { - super.beforeAll() - instances = Array( - Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), - Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), - Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) - ) - instancesConstantFeature = Array( - Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), - Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), - Instance(2.0, 0.3, Vectors.dense(1.0, 0.5)) - ) - instancesConstantFeatureFiltered = Array( - Instance(0.0, 0.1, Vectors.dense(2.0)), - Instance(1.0, 0.5, Vectors.dense(1.0)), - Instance(2.0, 0.3, Vectors.dense(0.5)) - ) - standardizedInstances = standardize(instances) - } - - /** Get summary statistics for some data and create a new LogisticAggregator. */ - private def getNewAggregator( - instances: Array[Instance], - coefficients: Vector, - fitIntercept: Boolean, - isMultinomial: Boolean): LogisticAggregator = { - val (featuresSummarizer, ySummarizer) = - Summarizer.getClassificationSummarizers(sc.parallelize(instances)) - val numClasses = ySummarizer.histogram.length - val featuresStd = featuresSummarizer.std.toArray - val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) - val bcCoefficients = spark.sparkContext.broadcast(coefficients) - new LogisticAggregator(bcFeaturesStd, numClasses, fitIntercept, isMultinomial)(bcCoefficients) - } - - /** Get summary statistics for some data and create a new BlockHingeAggregator. */ - private def getNewBlockAggregator( - instances: Array[Instance], - coefficients: Vector, - fitIntercept: Boolean, - multinomial: Boolean): BlockLogisticAggregator = { - val (_, ySummarizer) = - Summarizer.getClassificationSummarizers(sc.parallelize(instances)) - val numFeatures = instances.head.features.size - val numClasses = ySummarizer.histogram.length - val bcCoefficients = spark.sparkContext.broadcast(coefficients) - new BlockLogisticAggregator(numFeatures, numClasses, fitIntercept, multinomial)(bcCoefficients) - } - - test("aggregator add method input size") { - val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) - val interceptArray = Array(4.0, 2.0, -3.0) - val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), - fitIntercept = true, isMultinomial = true) - withClue("LogisticAggregator features dimension must match coefficients dimension") { - intercept[IllegalArgumentException] { - agg.add(Instance(1.0, 1.0, Vectors.dense(2.0))) - } - } - } - - test("negative weight") { - val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) - val interceptArray = Array(4.0, 2.0, -3.0) - val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), - fitIntercept = true, isMultinomial = true) - withClue("LogisticAggregator does not support negative instance weights") { - intercept[IllegalArgumentException] { - agg.add(Instance(1.0, -1.0, Vectors.dense(2.0, 1.0))) - } - } - } - - test("check sizes multinomial") { - val rng = new scala.util.Random - val numFeatures = instances.head.features.size - val numClasses = instances.map(_.label).toSet.size - val coefWithIntercept = Vectors.dense( - Array.fill(numClasses * (numFeatures + 1))(rng.nextDouble)) - val coefWithoutIntercept = Vectors.dense( - Array.fill(numClasses * numFeatures)(rng.nextDouble)) - val aggIntercept = getNewAggregator(instances, coefWithIntercept, fitIntercept = true, - isMultinomial = true) - val aggNoIntercept = getNewAggregator(instances, coefWithoutIntercept, fitIntercept = false, - isMultinomial = true) - instances.foreach(aggIntercept.add) - instances.foreach(aggNoIntercept.add) - - assert(aggIntercept.gradient.size === (numFeatures + 1) * numClasses) - assert(aggNoIntercept.gradient.size === numFeatures * numClasses) - } - - test("check sizes binomial") { - val rng = new scala.util.Random - val binaryInstances = instances.filter(_.label < 2.0) - val numFeatures = binaryInstances.head.features.size - val coefWithIntercept = Vectors.dense(Array.fill(numFeatures + 1)(rng.nextDouble)) - val coefWithoutIntercept = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble)) - val aggIntercept = getNewAggregator(binaryInstances, coefWithIntercept, fitIntercept = true, - isMultinomial = false) - val aggNoIntercept = getNewAggregator(binaryInstances, coefWithoutIntercept, - fitIntercept = false, isMultinomial = false) - binaryInstances.foreach(aggIntercept.add) - binaryInstances.foreach(aggNoIntercept.add) - - assert(aggIntercept.gradient.size === numFeatures + 1) - assert(aggNoIntercept.gradient.size === numFeatures) - } - - test("check correctness multinomial") { - /* - Check that the aggregator computes loss/gradient for: - -sum_i w_i * (beta_y dot x_i - log(sum_k e^(beta_k dot x_i))) - */ - val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) - val interceptArray = Array(4.0, 2.0, -3.0) - val numFeatures = instances.head.features.size - val numClasses = instances.map(_.label).toSet.size - val intercepts = Vectors.dense(interceptArray) - val (featuresSummarizer, ySummarizer) = - Summarizer.getClassificationSummarizers(sc.parallelize(instances)) - val featuresStd = featuresSummarizer.std.toArray - val weightSum = instances.map(_.weight).sum - - val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), - fitIntercept = true, isMultinomial = true) - instances.foreach(agg.add) - - // compute the loss - val stdCoef = coefArray.indices.map(i => coefArray(i) / featuresStd(i / numClasses)).toArray - val linearPredictors = instances.map { case Instance(l, w, f) => - val result = intercepts.copy.toDense - BLAS.gemv(1.0, Matrices.dense(numClasses, numFeatures, stdCoef), f, 1.0, result) - (l, w, result) - } - - // sum_i w * beta_k dot x_i - val sumLinear = linearPredictors.map { case (l, w, p) => - w * p(l.toInt) - }.sum - - // sum_i w * log(sum_k e^(beta_K dot x_i)) - val sumLogs = linearPredictors.map { case (l, w, p) => - w * math.log(p.values.map(math.exp).sum) - }.sum - val loss = (sumLogs - sumLinear) / weightSum - - - // compute the gradients - val gradientCoef = new Array[Double](numFeatures * numClasses) - val gradientIntercept = new Array[Double](numClasses) - instances.foreach { case Instance(l, w, f) => - val margin = intercepts.copy.toDense - BLAS.gemv(1.0, Matrices.dense(numClasses, numFeatures, stdCoef), f, 1.0, margin) - val sum = margin.values.map(math.exp).sum - - gradientCoef.indices.foreach { i => - val fStd = f(i / numClasses) / featuresStd(i / numClasses) - val cidx = i % numClasses - if (cidx == l.toInt) gradientCoef(i) -= w * fStd - gradientCoef(i) += w * math.exp(margin(cidx)) / sum * fStd - } - - gradientIntercept.indices.foreach { i => - val cidx = i % numClasses - if (cidx == l.toInt) gradientIntercept(i) -= w - gradientIntercept(i) += w * math.exp(margin(cidx)) / sum - } - } - val gradient = Vectors.dense((gradientCoef ++ gradientIntercept).map(_ / weightSum)) - - assert(loss ~== agg.loss relTol 0.01) - assert(gradient ~== agg.gradient relTol 0.01) - - Seq(1, 2, 4).foreach { blockSize => - val blocks1 = standardizedInstances - .grouped(blockSize) - .map(seq => InstanceBlock.fromInstances(seq)) - .toArray - val blocks2 = blocks1.map { block => - new InstanceBlock(block.labels, block.weights, block.matrix.toSparseRowMajor) - } - - Seq(blocks1, blocks2).foreach { blocks => - val blockAgg = getNewBlockAggregator(standardizedInstances, - Vectors.dense(coefArray ++ interceptArray), true, true) - blocks.foreach(blockAgg.add) - assert(agg.loss ~== blockAgg.loss relTol 1e-9) - assert(agg.gradient ~== blockAgg.gradient relTol 1e-9) - } - } - } - - test("check correctness binomial") { - /* - Check that the aggregator computes loss/gradient for: - -sum_i y_i * log(1 / (1 + e^(-beta dot x_i)) + (1 - y_i) * log(1 - 1 / (1 + e^(-beta dot x_i)) - */ - val binaryInstances = instances.map { instance => - if (instance.label <= 1.0) instance else Instance(0.0, instance.weight, instance.features) - } - val coefArray = Array(1.0, 2.0) - val intercept = 1.0 - val numFeatures = binaryInstances.head.features.size - val (featuresSummarizer, _) = - Summarizer.getClassificationSummarizers(sc.parallelize(binaryInstances)) - val featuresStd = featuresSummarizer.std.toArray - val weightSum = binaryInstances.map(_.weight).sum - - val agg = getNewAggregator(binaryInstances, Vectors.dense(coefArray ++ Array(intercept)), - fitIntercept = true, isMultinomial = false) - binaryInstances.foreach(agg.add) - - // compute the loss - val stdCoef = coefArray.indices.map(i => coefArray(i) / featuresStd(i)).toArray - val lossSum = binaryInstances.map { case Instance(l, w, f) => - val margin = BLAS.dot(Vectors.dense(stdCoef), f) + intercept - val prob = 1.0 / (1.0 + math.exp(-margin)) - -w * l * math.log(prob) - w * (1.0 - l) * math.log1p(-prob) - }.sum - val loss = lossSum / weightSum - - // compute the gradients - val gradientCoef = new Array[Double](numFeatures) - var gradientIntercept = 0.0 - binaryInstances.foreach { case Instance(l, w, f) => - val margin = BLAS.dot(f, Vectors.dense(coefArray)) + intercept - gradientCoef.indices.foreach { i => - gradientCoef(i) += w * (1.0 / (1.0 + math.exp(-margin)) - l) * f(i) / featuresStd(i) - } - gradientIntercept += w * (1.0 / (1.0 + math.exp(-margin)) - l) - } - val gradient = Vectors.dense((gradientCoef ++ Array(gradientIntercept)).map(_ / weightSum)) - - assert(loss ~== agg.loss relTol 0.01) - assert(gradient ~== agg.gradient relTol 0.01) - - Seq(1, 2, 4).foreach { blockSize => - val blocks1 = standardize(binaryInstances) - .grouped(blockSize) - .map(seq => InstanceBlock.fromInstances(seq)) - .toArray - val blocks2 = blocks1.map { block => - new InstanceBlock(block.labels, block.weights, block.matrix.toSparseRowMajor) - } - - Seq(blocks1, blocks2).foreach { blocks => - val blockAgg = getNewBlockAggregator(binaryInstances, - Vectors.dense(coefArray ++ Array(intercept)), true, false) - blocks.foreach(blockAgg.add) - assert(agg.loss ~== blockAgg.loss relTol 1e-9) - assert(agg.gradient ~== blockAgg.gradient relTol 1e-9) - } - } - } - - test("check with zero standard deviation") { - val binaryInstances = instancesConstantFeature.map { instance => - if (instance.label <= 1.0) instance else Instance(0.0, instance.weight, instance.features) - } - val binaryInstancesFiltered = instancesConstantFeatureFiltered.map { instance => - if (instance.label <= 1.0) instance else Instance(0.0, instance.weight, instance.features) - } - val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) - val coefArrayFiltered = Array(3.0, 0.0, -1.0) - val interceptArray = Array(4.0, 2.0, -3.0) - val aggConstantFeature = getNewAggregator(instancesConstantFeature, - Vectors.dense(coefArray ++ interceptArray), fitIntercept = true, isMultinomial = true) - val aggConstantFeatureFiltered = getNewAggregator(instancesConstantFeatureFiltered, - Vectors.dense(coefArrayFiltered ++ interceptArray), fitIntercept = true, isMultinomial = true) - - instancesConstantFeature.foreach(aggConstantFeature.add) - instancesConstantFeatureFiltered.foreach(aggConstantFeatureFiltered.add) - - // constant features should not affect gradient - def validateGradient(grad: Vector, gradFiltered: Vector, numCoefficientSets: Int): Unit = { - for (i <- 0 until numCoefficientSets) { - assert(grad(i) === 0.0) - assert(grad(numCoefficientSets + i) == gradFiltered(i)) - } - } - - validateGradient(aggConstantFeature.gradient, aggConstantFeatureFiltered.gradient, 3) - - val binaryCoefArray = Array(1.0, 2.0) - val binaryCoefArrayFiltered = Array(2.0) - val intercept = 1.0 - val aggConstantFeatureBinary = getNewAggregator(binaryInstances, - Vectors.dense(binaryCoefArray ++ Array(intercept)), fitIntercept = true, - isMultinomial = false) - val aggConstantFeatureBinaryFiltered = getNewAggregator(binaryInstancesFiltered, - Vectors.dense(binaryCoefArrayFiltered ++ Array(intercept)), fitIntercept = true, - isMultinomial = false) - binaryInstances.foreach(aggConstantFeatureBinary.add) - binaryInstancesFiltered.foreach(aggConstantFeatureBinaryFiltered.add) - - // constant features should not affect gradient - validateGradient(aggConstantFeatureBinary.gradient, - aggConstantFeatureBinaryFiltered.gradient, 1) - } -} diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregatorSuite.scala new file mode 100644 index 0000000000000..d00fdaca15828 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregatorSuite.scala @@ -0,0 +1,387 @@ +/* + * 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.ml.optim.aggregator + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.stat.Summarizer +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class MultinomialLogisticBlockAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { + + @transient var instances: Array[Instance] = _ + @transient var instancesConstantFeature: Array[Instance] = _ + @transient var instancesConstantFeatureFiltered: Array[Instance] = _ + @transient var scaledInstances: Array[Instance] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + instances = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), + Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) + ) + instancesConstantFeature = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), + Instance(2.0, 0.3, Vectors.dense(1.0, 0.5)) + ) + instancesConstantFeatureFiltered = Array( + Instance(0.0, 0.1, Vectors.dense(2.0)), + Instance(1.0, 0.5, Vectors.dense(1.0)), + Instance(2.0, 0.3, Vectors.dense(0.5)) + ) + scaledInstances = standardize(instances) + } + + /** Get summary statistics for some data and create a new MultinomialLogisticBlockAggregator. */ + private def getNewAggregator( + instances: Array[Instance], + coefficients: Vector, + fitIntercept: Boolean, + fitWithMean: Boolean): MultinomialLogisticBlockAggregator = { + val (featuresSummarizer, _) = + Summarizer.getClassificationSummarizers(sc.parallelize(instances)) + val featuresStd = featuresSummarizer.std.toArray + val featuresMean = featuresSummarizer.mean.toArray + val inverseStd = featuresStd.map(std => if (std != 0) 1.0 / std else 0.0) + val scaledMean = inverseStd.zip(featuresMean).map(t => t._1 * t._2) + val bcInverseStd = sc.broadcast(inverseStd) + val bcScaledMean = sc.broadcast(scaledMean) + val bcCoefficients = sc.broadcast(coefficients) + new MultinomialLogisticBlockAggregator(bcInverseStd, + bcScaledMean, fitIntercept, fitWithMean)(bcCoefficients) + } + + test("sparse coefficients") { + val bcInverseStd = sc.broadcast(Array(1.0)) + val bcScaledMean = sc.broadcast(Array(2.0)) + val bcCoefficients = sc.broadcast(Vectors.sparse(4, Array(0), Array(1.0))) + val binaryAgg = new MultinomialLogisticBlockAggregator(bcInverseStd, bcScaledMean, + fitIntercept = true, fitWithMean = false)(bcCoefficients) + val block = InstanceBlock.fromInstances(Seq(Instance(1.0, 1.0, Vectors.dense(1.0)))) + val thrownBinary = withClue("aggregator cannot handle sparse coefficients") { + intercept[IllegalArgumentException] { + binaryAgg.add(block) + } + } + assert(thrownBinary.getMessage.contains("coefficients only supports dense")) + } + + test("aggregator add method input size") { + val coefArray = Array(1.0, 2.0, 3.0, 4.0, 5.0, 6.0) + val interceptArray = Array(7.0, 8.0, 9.0) + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), + fitIntercept = true, fitWithMean = true) + val block = InstanceBlock.fromInstances(Seq(Instance(1.0, 1.0, Vectors.dense(2.0)))) + withClue("BinaryLogisticBlockAggregator features dimension must match coefficients dimension") { + intercept[IllegalArgumentException] { + agg.add(block) + } + } + } + + test("negative weight") { + val coefArray = Array(1.0, 2.0, 3.0, 4.0, 5.0, 6.0) + val interceptArray = Array(7.0, 8.0, 9.0) + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), + fitIntercept = true, fitWithMean = true) + val block = InstanceBlock.fromInstances(Seq(Instance(1.0, -1.0, Vectors.dense(2.0, 1.0)))) + withClue("BinaryLogisticBlockAggregator does not support negative instance weights") { + intercept[IllegalArgumentException] { + agg.add(block) + } + } + } + + test("check sizes") { + val rng = new scala.util.Random + val numFeatures = instances.head.features.size + val numClasses = instances.map(_.label).distinct.size + val coefWithIntercept = Vectors.dense( + Array.fill(numClasses * (numFeatures + 1))(rng.nextDouble)) + val coefWithoutIntercept = Vectors.dense( + Array.fill(numClasses * numFeatures)(rng.nextDouble)) + val block = InstanceBlock.fromInstances(instances) + + val aggIntercept = getNewAggregator(instances, coefWithIntercept, + fitIntercept = true, fitWithMean = false) + aggIntercept.add(block) + assert(aggIntercept.gradient.size === (numFeatures + 1) * numClasses) + + val aggNoIntercept = getNewAggregator(instances, coefWithoutIntercept, + fitIntercept = false, fitWithMean = false) + aggNoIntercept.add(block) + assert(aggNoIntercept.gradient.size === numFeatures * numClasses) + } + + test("check correctness: fitIntercept = false") { + val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) + val numFeatures = instances.head.features.size + val numClasses = instances.map(_.label).toSet.size + val (featuresSummarizer, _) = + Summarizer.getClassificationSummarizers(sc.parallelize(instances)) + val featuresStd = featuresSummarizer.std + val stdCoefMat = Matrices.dense(numClasses, numFeatures, + Array.tabulate(coefArray.size)(i => coefArray(i) / featuresStd(i / numClasses))) + val weightSum = instances.map(_.weight).sum + + // compute the loss + val linearPredictors = instances.map { case Instance(l, w, f) => + val result = new DenseVector(Array.ofDim[Double](numClasses)) + BLAS.gemv(1.0, stdCoefMat, f, 1.0, result) + (l, w, result) + } + + // sum_i w * beta_k dot x_i + val sumLinear = linearPredictors.map { case (l, w, p) => + w * p(l.toInt) + }.sum + + // sum_i w * log(sum_k e^(beta_K dot x_i)) + val sumLogs = linearPredictors.map { case (l, w, p) => + w * math.log(p.values.map(math.exp).sum) + }.sum + val loss = (sumLogs - sumLinear) / weightSum + + // compute the gradients + val gradientCoef = new Array[Double](numFeatures * numClasses) + instances.foreach { case Instance(l, w, f) => + val margin = new DenseVector(Array.ofDim[Double](numClasses)) + BLAS.gemv(1.0, stdCoefMat, f, 1.0, margin) + val sum = margin.values.map(math.exp).sum + + gradientCoef.indices.foreach { i => + val fStd = f(i / numClasses) / featuresStd(i / numClasses) + val cidx = i % numClasses + if (cidx == l.toInt) gradientCoef(i) -= w * fStd + gradientCoef(i) += w * math.exp(margin(cidx)) / sum * fStd + } + } + val gradient = Vectors.dense((gradientCoef).map(_ / weightSum)) + + Seq(1, 2, 4).foreach { blockSize => + val blocks1 = scaledInstances + .grouped(blockSize) + .map(seq => InstanceBlock.fromInstances(seq)) + .toArray + val blocks2 = blocks1.map { block => + new InstanceBlock(block.labels, block.weights, block.matrix.toSparseRowMajor) + } + + Seq(blocks1, blocks2).foreach { blocks => + val agg = getNewAggregator(instances, Vectors.dense(coefArray), + fitIntercept = false, fitWithMean = false) + blocks.foreach(agg.add) + assert(agg.loss ~== loss relTol 1e-9) + assert(agg.gradient ~== gradient relTol 1e-9) + } + } + } + + test("check correctness: fitIntercept = true, fitWithMean = false") { + val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) + val interceptArray = Array(4.0, 2.0, -3.0) + val numFeatures = instances.head.features.size + val numClasses = instances.map(_.label).toSet.size + val intercepts = Vectors.dense(interceptArray) + val (featuresSummarizer, _) = + Summarizer.getClassificationSummarizers(sc.parallelize(instances)) + val featuresStd = featuresSummarizer.std + val stdCoefMat = Matrices.dense(numClasses, numFeatures, + Array.tabulate(coefArray.size)(i => coefArray(i) / featuresStd(i / numClasses))) + val weightSum = instances.map(_.weight).sum + + // compute the loss + val linearPredictors = instances.map { case Instance(l, w, f) => + val result = intercepts.copy.toDense + BLAS.gemv(1.0, stdCoefMat, f, 1.0, result) + (l, w, result) + } + + // sum_i w * beta_k dot x_i + val sumLinear = linearPredictors.map { case (l, w, p) => + w * p(l.toInt) + }.sum + + // sum_i w * log(sum_k e^(beta_K dot x_i)) + val sumLogs = linearPredictors.map { case (l, w, p) => + w * math.log(p.values.map(math.exp).sum) + }.sum + val loss = (sumLogs - sumLinear) / weightSum + + // compute the gradients + val gradientCoef = new Array[Double](numFeatures * numClasses) + val gradientIntercept = new Array[Double](numClasses) + instances.foreach { case Instance(l, w, f) => + val margin = intercepts.copy.toDense + BLAS.gemv(1.0, stdCoefMat, f, 1.0, margin) + val sum = margin.values.map(math.exp).sum + + gradientCoef.indices.foreach { i => + val fStd = f(i / numClasses) / featuresStd(i / numClasses) + val cidx = i % numClasses + if (cidx == l.toInt) gradientCoef(i) -= w * fStd + gradientCoef(i) += w * math.exp(margin(cidx)) / sum * fStd + } + + gradientIntercept.indices.foreach { i => + val cidx = i % numClasses + if (cidx == l.toInt) gradientIntercept(i) -= w + gradientIntercept(i) += w * math.exp(margin(cidx)) / sum + } + } + val gradient = Vectors.dense((gradientCoef ++ gradientIntercept).map(_ / weightSum)) + + Seq(1, 2, 4).foreach { blockSize => + val blocks1 = scaledInstances + .grouped(blockSize) + .map(seq => InstanceBlock.fromInstances(seq)) + .toArray + val blocks2 = blocks1.map { block => + new InstanceBlock(block.labels, block.weights, block.matrix.toSparseRowMajor) + } + + Seq(blocks1, blocks2).foreach { blocks => + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), + fitIntercept = true, fitWithMean = false) + blocks.foreach(agg.add) + assert(agg.loss ~== loss relTol 1e-9) + assert(agg.gradient ~== gradient relTol 1e-9) + } + } + } + + test("check correctness: fitIntercept = true, fitWithMean = true") { + val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) + val interceptArray = Array(4.0, 2.0, -3.0) + val numFeatures = instances.head.features.size + val numClasses = instances.map(_.label).toSet.size + val intercepts = Vectors.dense(interceptArray) + val (featuresSummarizer, _) = + Summarizer.getClassificationSummarizers(sc.parallelize(instances)) + val featuresStd = featuresSummarizer.std + val featuresMean = featuresSummarizer.mean + val stdCoefMat = Matrices.dense(numClasses, numFeatures, + Array.tabulate(coefArray.size)(i => coefArray(i) / featuresStd(i / numClasses))) + val weightSum = instances.map(_.weight).sum + + // compute the loss + val linearPredictors = instances.map { case Instance(l, w, f) => + val centered = f.toDense.copy + BLAS.axpy(-1.0, featuresMean, centered) + val result = intercepts.copy.toDense + BLAS.gemv(1.0, stdCoefMat, centered, 1.0, result) + (l, w, result) + } + + // sum_i w * beta_k dot x_i + val sumLinear = linearPredictors.map { case (l, w, p) => + w * p(l.toInt) + }.sum + + // sum_i w * log(sum_k e^(beta_K dot x_i)) + val sumLogs = linearPredictors.map { case (l, w, p) => + w * math.log(p.values.map(math.exp).sum) + }.sum + val loss = (sumLogs - sumLinear) / weightSum + + // compute the gradients + val gradientCoef = new Array[Double](numFeatures * numClasses) + val gradientIntercept = new Array[Double](numClasses) + instances.foreach { case Instance(l, w, f) => + val centered = f.toDense.copy + BLAS.axpy(-1.0, featuresMean, centered) + val margin = intercepts.copy.toDense + BLAS.gemv(1.0, stdCoefMat, centered, 1.0, margin) + val sum = margin.values.map(math.exp).sum + + gradientCoef.indices.foreach { i => + val fStd = centered(i / numClasses) / featuresStd(i / numClasses) + val cidx = i % numClasses + if (cidx == l.toInt) gradientCoef(i) -= w * fStd + gradientCoef(i) += w * math.exp(margin(cidx)) / sum * fStd + } + + gradientIntercept.indices.foreach { i => + val cidx = i % numClasses + if (cidx == l.toInt) gradientIntercept(i) -= w + gradientIntercept(i) += w * math.exp(margin(cidx)) / sum + } + } + val gradient = Vectors.dense((gradientCoef ++ gradientIntercept).map(_ / weightSum)) + + Seq(1, 2, 4).foreach { blockSize => + val blocks1 = scaledInstances + .grouped(blockSize) + .map(seq => InstanceBlock.fromInstances(seq)) + .toArray + val blocks2 = blocks1.map { block => + new InstanceBlock(block.labels, block.weights, block.matrix.toSparseRowMajor) + } + + Seq(blocks1, blocks2).foreach { blocks => + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), + fitIntercept = true, fitWithMean = true) + blocks.foreach(agg.add) + assert(agg.loss ~== loss relTol 1e-9) + assert(agg.gradient ~== gradient relTol 1e-9) + } + } + } + + test("check with zero standard deviation") { + val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) + val coefArrayFiltered = Array(3.0, 0.0, -1.0) + val interceptArray = Array(4.0, 2.0, -3.0) + + Seq((false, false), (true, false), (true, true)).foreach { case (fitIntercept, fitWithMean) => + val coefVec = if (fitIntercept) { + Vectors.dense(coefArray ++ interceptArray) + } else { + Vectors.dense(coefArray) + } + val aggConstantFeature = getNewAggregator(instancesConstantFeature, + coefVec, fitIntercept = fitIntercept, fitWithMean = fitWithMean) + aggConstantFeature + .add(InstanceBlock.fromInstances(standardize(instancesConstantFeature))) + val grad = aggConstantFeature.gradient + + val coefVecFiltered = if (fitIntercept) { + Vectors.dense(coefArrayFiltered ++ interceptArray) + } else { + Vectors.dense(coefArrayFiltered) + } + val aggConstantFeatureFiltered = getNewAggregator(instancesConstantFeatureFiltered, + coefVecFiltered, fitIntercept = fitIntercept, fitWithMean = fitWithMean) + aggConstantFeatureFiltered + .add(InstanceBlock.fromInstances(standardize(instancesConstantFeatureFiltered))) + val gradFiltered = aggConstantFeatureFiltered.gradient + + // constant features should not affect gradient + assert(Vectors.dense(grad.toArray.take(3)) === Vectors.zeros(3)) + assert(Vectors.dense(grad.toArray.slice(3, 6)) ~== + Vectors.dense(gradFiltered.toArray.take(3)) relTol 1e-9) + if (fitIntercept) { + assert(Vectors.dense(grad.toArray.takeRight(3)) ~== + Vectors.dense(gradFiltered.toArray.takeRight(3)) relTol 1e-9) + } + } + } +} From 150769bcedb6e4a97596e0f04d686482cd09e92a Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 25 Mar 2021 08:31:57 +0900 Subject: [PATCH 22/24] [SPARK-34833][SQL] Apply right-padding correctly for correlated subqueries ### What changes were proposed in this pull request? This PR intends to fix the bug that does not apply right-padding for char types inside correlated subquries. For example, a query below returns nothing in master, but a correct result is `c`. ``` scala> sql(s"CREATE TABLE t1(v VARCHAR(3), c CHAR(5)) USING parquet") scala> sql(s"CREATE TABLE t2(v VARCHAR(5), c CHAR(7)) USING parquet") scala> sql("INSERT INTO t1 VALUES ('c', 'b')") scala> sql("INSERT INTO t2 VALUES ('a', 'b')") scala> val df = sql(""" |SELECT v FROM t1 |WHERE 'a' IN (SELECT v FROM t2 WHERE t2.c = t1.c )""".stripMargin) scala> df.show() +---+ | v| +---+ +---+ ``` This is because `ApplyCharTypePadding` does not handle the case above to apply right-padding into `'abc'`. This PR modifies the code in `ApplyCharTypePadding` for handling it correctly. ``` // Before this PR: scala> df.explain(true) == Analyzed Logical Plan == v: string Project [v#13] +- Filter a IN (list#12 [c#14]) : +- Project [v#15] : +- Filter (c#16 = outer(c#14)) : +- SubqueryAlias spark_catalog.default.t2 : +- Relation default.t2[v#15,c#16] parquet +- SubqueryAlias spark_catalog.default.t1 +- Relation default.t1[v#13,c#14] parquet scala> df.show() +---+ | v| +---+ +---+ // After this PR: scala> df.explain(true) == Analyzed Logical Plan == v: string Project [v#43] +- Filter a IN (list#42 [c#44]) : +- Project [v#45] : +- Filter (c#46 = rpad(outer(c#44), 7, )) : +- SubqueryAlias spark_catalog.default.t2 : +- Relation default.t2[v#45,c#46] parquet +- SubqueryAlias spark_catalog.default.t1 +- Relation default.t1[v#43,c#44] parquet scala> df.show() +---+ | v| +---+ | c| +---+ ``` This fix is lated to TPCDS q17; the query returns nothing because of this bug: https://github.com/apache/spark/pull/31886/files#r599333799 ### Why are the changes needed? Bugfix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests added. Closes #31940 from maropu/FixCharPadding. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../sql/catalyst/analysis/Analyzer.scala | 45 ++++++++++++--- .../spark/sql/CharVarcharTestSuite.scala | 57 ++++++++++++++----- 2 files changed, 79 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 947aeb56f6ca2..f5f0f5c2037a8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -4008,16 +4008,28 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = { plan.resolveOperatorsUp { - case operator if operator.resolved => operator.transformExpressionsUp { + case operator => operator.transformExpressionsUp { + case e if !e.childrenResolved => e + // String literal is treated as char type when it's compared to a char type column. // We should pad the shorter one to the longer length. case b @ BinaryComparison(attr: Attribute, lit) if lit.foldable => - padAttrLitCmp(attr, lit).map { newChildren => + padAttrLitCmp(attr, attr.metadata, lit).map { newChildren => b.withNewChildren(newChildren) }.getOrElse(b) case b @ BinaryComparison(lit, attr: Attribute) if lit.foldable => - padAttrLitCmp(attr, lit).map { newChildren => + padAttrLitCmp(attr, attr.metadata, lit).map { newChildren => + b.withNewChildren(newChildren.reverse) + }.getOrElse(b) + + case b @ BinaryComparison(or @ OuterReference(attr: Attribute), lit) if lit.foldable => + padAttrLitCmp(or, attr.metadata, lit).map { newChildren => + b.withNewChildren(newChildren) + }.getOrElse(b) + + case b @ BinaryComparison(lit, or @ OuterReference(attr: Attribute)) if lit.foldable => + padAttrLitCmp(or, attr.metadata, lit).map { newChildren => b.withNewChildren(newChildren.reverse) }.getOrElse(b) @@ -4041,6 +4053,12 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { case b @ BinaryComparison(left: Attribute, right: Attribute) => b.withNewChildren(CharVarcharUtils.addPaddingInStringComparison(Seq(left, right))) + case b @ BinaryComparison(OuterReference(left: Attribute), right: Attribute) => + b.withNewChildren(padOuterRefAttrCmp(left, right)) + + case b @ BinaryComparison(left: Attribute, OuterReference(right: Attribute)) => + b.withNewChildren(padOuterRefAttrCmp(right, left).reverse) + case i @ In(attr: Attribute, list) if list.forall(_.isInstanceOf[Attribute]) => val newChildren = CharVarcharUtils.addPaddingInStringComparison( attr +: list.map(_.asInstanceOf[Attribute])) @@ -4049,9 +4067,12 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { } } - private def padAttrLitCmp(attr: Attribute, lit: Expression): Option[Seq[Expression]] = { - if (attr.dataType == StringType) { - CharVarcharUtils.getRawType(attr.metadata).flatMap { + private def padAttrLitCmp( + expr: Expression, + metadata: Metadata, + lit: Expression): Option[Seq[Expression]] = { + if (expr.dataType == StringType) { + CharVarcharUtils.getRawType(metadata).flatMap { case CharType(length) => val str = lit.eval().asInstanceOf[UTF8String] if (str == null) { @@ -4059,9 +4080,9 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { } else { val stringLitLen = str.numChars() if (length < stringLitLen) { - Some(Seq(StringRPad(attr, Literal(stringLitLen)), lit)) + Some(Seq(StringRPad(expr, Literal(stringLitLen)), lit)) } else if (length > stringLitLen) { - Some(Seq(attr, StringRPad(lit, Literal(length)))) + Some(Seq(expr, StringRPad(lit, Literal(length)))) } else { None } @@ -4073,6 +4094,14 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { } } + private def padOuterRefAttrCmp(outerAttr: Attribute, attr: Attribute): Seq[Expression] = { + val Seq(r, newAttr) = CharVarcharUtils.addPaddingInStringComparison(Seq(outerAttr, attr)) + val newOuterRef = r.transform { + case ar: Attribute if ar.semanticEquals(outerAttr) => OuterReference(ar) + } + Seq(newOuterRef, newAttr) + } + private def addPadding(expr: Expression, charLength: Int, targetLength: Int): Expression = { if (targetLength > charLength) StringRPad(expr, Literal(targetLength)) else expr } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala index 1e561747b6157..7d1e4ff040503 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala @@ -582,21 +582,6 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { } } - test("SPARK-33992: char/varchar resolution in correlated sub query") { - withTable("t1", "t2") { - sql(s"CREATE TABLE t1(v VARCHAR(3), c CHAR(5)) USING $format") - sql(s"CREATE TABLE t2(v VARCHAR(3), c CHAR(5)) USING $format") - sql("INSERT INTO t1 VALUES ('c', 'b')") - sql("INSERT INTO t2 VALUES ('a', 'b')") - - checkAnswer(sql( - """ - |SELECT v FROM t1 - |WHERE 'a' IN (SELECT v FROM t2 WHERE t1.c = t2.c )""".stripMargin), - Row("c")) - } - } - test("SPARK-34003: fix char/varchar fails w/ both group by and order by ") { withTable("t") { sql(s"CREATE TABLE t(v VARCHAR(3), i INT) USING $format") @@ -631,6 +616,48 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { checkAnswer(spark.table("t"), Row("c ")) } } + + test("SPARK-34833: right-padding applied correctly for correlated subqueries - join keys") { + withTable("t1", "t2") { + sql(s"CREATE TABLE t1(v VARCHAR(3), c CHAR(5)) USING $format") + sql(s"CREATE TABLE t2(v VARCHAR(5), c CHAR(8)) USING $format") + sql("INSERT INTO t1 VALUES ('c', 'b')") + sql("INSERT INTO t2 VALUES ('a', 'b')") + Seq("t1.c = t2.c", "t2.c = t1.c", + "t1.c = 'b'", "'b' = t1.c", "t1.c = 'b '", "'b ' = t1.c", + "t1.c = 'b '", "'b ' = t1.c").foreach { predicate => + checkAnswer(sql( + s""" + |SELECT v FROM t1 + |WHERE 'a' IN (SELECT v FROM t2 WHERE $predicate) + """.stripMargin), + Row("c")) + } + } + } + + test("SPARK-34833: right-padding applied correctly for correlated subqueries - other preds") { + withTable("t") { + sql(s"CREATE TABLE t(c0 INT, c1 CHAR(5), c2 CHAR(7)) USING $format") + sql("INSERT INTO t VALUES (1, 'abc', 'abc')") + Seq("c1 = 'abc'", "'abc' = c1", "c1 = 'abc '", "'abc ' = c1", + "c1 = 'abc '", "'abc ' = c1", "c1 = c2", "c2 = c1", + "c1 IN ('xxx', 'abc', 'xxxxx')", "c1 IN ('xxx', 'abc ', 'xxxxx')", + "c1 IN ('xxx', 'abc ', 'xxxxx')", + "c1 IN (c2)", "c2 IN (c1)").foreach { predicate => + checkAnswer(sql( + s""" + |SELECT c0 FROM t t1 + |WHERE ( + | SELECT count(*) AS c + | FROM t + | WHERE c0 = t1.c0 AND $predicate + |) > 0 + """.stripMargin), + Row(1)) + } + } + } } // Some basic char/varchar tests which doesn't rely on table implementation. From 9d561e6b5eb9f5f346dc0c4f3faba70148ab171a Mon Sep 17 00:00:00 2001 From: ulysses-you Date: Thu, 25 Mar 2021 10:21:44 +0800 Subject: [PATCH 23/24] [SPARK-34852][SQL] Close Hive session state should use withHiveState ### What changes were proposed in this pull request? Wrap Hive sessionStae `close` with `withHiveState` ### Why are the changes needed? Some reason: 1. Shutdown hook is invoked using different thread 2. Hive may use metasotre client again during closing Otherwise, we may get such expcetion with custom hive metastore version ``` 21/03/24 13:26:18 INFO session.SessionState: Failed to remove classloaders from DataNucleus java.lang.RuntimeException: Unable to instantiate org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient at org.apache.hadoop.hive.metastore.MetaStoreUtils.newInstance(MetaStoreUtils.java:1654) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.(RetryingMetaStoreClient.java:80) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:130) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:101) at org.apache.hadoop.hive.ql.metadata.Hive.createMetaStoreClient(Hive.java:3367) at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3406) at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3386) at org.apache.hadoop.hive.ql.session.SessionState.unCacheDataNucleusClassLoaders(SessionState.java:1546) at org.apache.hadoop.hive.ql.session.SessionState.close(SessionState.java:1536) at org.apache.spark.sql.hive.client.HiveClientImpl.closeState(HiveClientImpl.scala:172) at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$new$1(HiveClientImpl.scala:175) at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:214) at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$2(ShutdownHookManager.scala:188) ``` ### Does this PR introduce _any_ user-facing change? No, since this not released. ### How was this patch tested? manual test. Closes #31949 from ulysses-you/SPARK-34852. Authored-by: ulysses-you Signed-off-by: Kent Yao --- .../scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 35dd2c1bb4698..3658e382fd795 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -155,7 +155,7 @@ private[hive] class HiveClientImpl( } } - private def closeState(): Unit = { + private def closeState(): Unit = withHiveState { // These temp files are registered in o.a.h.u.ShutdownHookManager too during state start. // The state.close() will delete them if they are not null and try remove them from the // o.a.h.u.ShutdownHookManager which causes undesirable IllegalStateException. From 7838f55ca795ca222541de7bc3cb065205718957 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 25 Mar 2021 12:31:08 +0900 Subject: [PATCH 24/24] Revert "[SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes" This reverts commit 84df54b4951e4a78ea971c14124a8793dd5ac62d. --- .../q34.sf100/explain.txt | 4 ++-- .../approved-plans-modified/q34/explain.txt | 4 ++-- .../q53.sf100/explain.txt | 6 ++--- .../approved-plans-modified/q53/explain.txt | 6 ++--- .../q63.sf100/explain.txt | 6 ++--- .../approved-plans-modified/q63/explain.txt | 6 ++--- .../q7.sf100/explain.txt | 4 ++-- .../approved-plans-modified/q7/explain.txt | 4 ++-- .../q73.sf100/explain.txt | 4 ++-- .../approved-plans-modified/q73/explain.txt | 4 ++-- .../q89.sf100/explain.txt | 6 ++--- .../approved-plans-modified/q89/explain.txt | 6 ++--- .../q98.sf100/explain.txt | 6 ++--- .../approved-plans-modified/q98/explain.txt | 6 ++--- .../approved-plans-v1_4/q12.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q12/explain.txt | 6 ++--- .../approved-plans-v1_4/q13.sf100/explain.txt | 8 +++---- .../approved-plans-v1_4/q13/explain.txt | 8 +++---- .../approved-plans-v1_4/q16.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q16/explain.txt | 2 +- .../approved-plans-v1_4/q17.sf100/explain.txt | 8 +++---- .../approved-plans-v1_4/q17/explain.txt | 8 +++---- .../approved-plans-v1_4/q18.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q18/explain.txt | 4 ++-- .../approved-plans-v1_4/q20.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q20/explain.txt | 6 ++--- .../approved-plans-v1_4/q21.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q21/explain.txt | 2 +- .../q24a.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q24a/explain.txt | 4 ++-- .../q24b.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q24b/explain.txt | 4 ++-- .../approved-plans-v1_4/q26.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q26/explain.txt | 4 ++-- .../approved-plans-v1_4/q27.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q27/explain.txt | 4 ++-- .../approved-plans-v1_4/q32.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q32/explain.txt | 2 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q33/explain.txt | 4 ++-- .../approved-plans-v1_4/q34.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q34/explain.txt | 4 ++-- .../approved-plans-v1_4/q37.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q37/explain.txt | 2 +- .../approved-plans-v1_4/q38.sf100/explain.txt | 24 +++++++++---------- .../approved-plans-v1_4/q38/explain.txt | 12 +++++----- .../q39a.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q39a/explain.txt | 4 ++-- .../q39b.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q39b/explain.txt | 4 ++-- .../approved-plans-v1_4/q41.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q41/explain.txt | 4 ++-- .../approved-plans-v1_4/q44.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q44/explain.txt | 4 ++-- .../approved-plans-v1_4/q48.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q48/explain.txt | 6 ++--- .../approved-plans-v1_4/q5.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q5/explain.txt | 2 +- .../approved-plans-v1_4/q53.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q53/explain.txt | 6 ++--- .../approved-plans-v1_4/q54.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q54/explain.txt | 4 ++-- .../approved-plans-v1_4/q58.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q58/explain.txt | 2 +- .../approved-plans-v1_4/q63.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q63/explain.txt | 6 ++--- .../approved-plans-v1_4/q64.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q64/explain.txt | 4 ++-- .../approved-plans-v1_4/q67.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q67/explain.txt | 2 +- .../approved-plans-v1_4/q7.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q7/explain.txt | 4 ++-- .../approved-plans-v1_4/q70.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q70/explain.txt | 2 +- .../approved-plans-v1_4/q71.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q71/explain.txt | 4 ++-- .../approved-plans-v1_4/q72.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q72/explain.txt | 4 ++-- .../approved-plans-v1_4/q73.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q73/explain.txt | 4 ++-- .../approved-plans-v1_4/q75.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q75/explain.txt | 4 ++-- .../approved-plans-v1_4/q80.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q80/explain.txt | 2 +- .../approved-plans-v1_4/q82.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q82/explain.txt | 2 +- .../approved-plans-v1_4/q85.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q85/explain.txt | 6 ++--- .../approved-plans-v1_4/q87.sf100/explain.txt | 24 +++++++++---------- .../approved-plans-v1_4/q87/explain.txt | 12 +++++----- .../approved-plans-v1_4/q89.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q89/explain.txt | 6 ++--- .../approved-plans-v1_4/q91.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q91/explain.txt | 4 ++-- .../approved-plans-v1_4/q92.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q92/explain.txt | 2 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q93/explain.txt | 4 ++-- .../approved-plans-v1_4/q94.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q94/explain.txt | 6 ++--- .../approved-plans-v1_4/q95.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q95/explain.txt | 6 ++--- .../approved-plans-v1_4/q98.sf100/explain.txt | 6 ++--- .../approved-plans-v1_4/q98/explain.txt | 6 ++--- .../approved-plans-v2_7/q12.sf100/explain.txt | 6 ++--- .../approved-plans-v2_7/q12/explain.txt | 6 ++--- .../approved-plans-v2_7/q20.sf100/explain.txt | 6 ++--- .../approved-plans-v2_7/q20/explain.txt | 6 ++--- .../approved-plans-v2_7/q24.sf100/explain.txt | 4 ++-- .../approved-plans-v2_7/q24/explain.txt | 4 ++-- .../approved-plans-v2_7/q34.sf100/explain.txt | 4 ++-- .../approved-plans-v2_7/q34/explain.txt | 4 ++-- .../approved-plans-v2_7/q5a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q5a/explain.txt | 2 +- .../approved-plans-v2_7/q64.sf100/explain.txt | 4 ++-- .../approved-plans-v2_7/q64/explain.txt | 4 ++-- .../q67a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q67a/explain.txt | 2 +- .../q70a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q70a/explain.txt | 2 +- .../approved-plans-v2_7/q72.sf100/explain.txt | 4 ++-- .../approved-plans-v2_7/q72/explain.txt | 4 ++-- .../approved-plans-v2_7/q75.sf100/explain.txt | 4 ++-- .../approved-plans-v2_7/q75/explain.txt | 4 ++-- .../q80a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q80a/explain.txt | 2 +- .../approved-plans-v2_7/q98.sf100/explain.txt | 6 ++--- .../approved-plans-v2_7/q98/explain.txt | 6 ++--- .../apache/spark/sql/PlanStabilitySuite.scala | 18 +++++++------- 129 files changed, 306 insertions(+), 306 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt index ebc6009080bfa..ff33c0b00120b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt index cd556c0bccb46..5ebb8e180e1e0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt index a3223d60c96be..92db706d594f2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt index 172e66022046d..8b29f7c168765 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt index 75cdaecea7595..8e0aaf2c479b8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt index c8e0821dfacda..712ba76f2878a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt index daa0ca7b16562..fe4db3541cbd9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt @@ -116,7 +116,7 @@ Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_p Output [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -124,7 +124,7 @@ Input [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_stat (20) Filter [codegen id : 3] Input [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20] -Condition : ((((((isnotnull(cd_gender#18) AND isnotnull(cd_marital_status#19)) AND isnotnull(cd_education_status#20)) AND (cd_gender#18 = F)) AND (cd_marital_status#19 = W)) AND (cd_education_status#20 = Primary )) AND isnotnull(cd_demo_sk#17)) +Condition : ((((((isnotnull(cd_gender#18) AND isnotnull(cd_marital_status#19)) AND isnotnull(cd_education_status#20)) AND (cd_gender#18 = F)) AND (cd_marital_status#19 = W)) AND (cd_education_status#20 = Primary)) AND isnotnull(cd_demo_sk#17)) (21) Project [codegen id : 3] Output [1]: [cd_demo_sk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt index a9da9118323b6..c1f0c1bd01451 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt index c2909d9f5531f..9dc7b54bf5bb6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt @@ -60,7 +60,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint) Output [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (6) ColumnarToRow @@ -68,7 +68,7 @@ Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#1 (7) Filter Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] -Condition : (((((isnotnull(hd_vehicle_count#10) AND isnotnull(hd_dep_count#9)) AND ((hd_buy_potential#8 = >10000 ) OR (hd_buy_potential#8 = Unknown ))) AND (hd_vehicle_count#10 > 0)) AND ((cast(hd_dep_count#9 as double) / cast(hd_vehicle_count#10 as double)) > 1.0)) AND isnotnull(hd_demo_sk#7)) +Condition : (((((isnotnull(hd_vehicle_count#10) AND isnotnull(hd_dep_count#9)) AND ((hd_buy_potential#8 = >10000) OR (hd_buy_potential#8 = Unknown))) AND (hd_vehicle_count#10 > 0)) AND ((cast(hd_dep_count#9 as double) / cast(hd_vehicle_count#10 as double)) > 1.0)) AND isnotnull(hd_demo_sk#7)) (8) Project Output [1]: [hd_demo_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt index 5ec772ca3d638..559ef93fb70ac 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt index a325bf820a4f9..393b6a4da4255 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt @@ -109,7 +109,7 @@ Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, d_moy#8, s_store_sk#1 Output [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Home ,Books ,Electronics ]),In(i_class, [wallpaper ,parenting ,musical ])),And(In(i_category, [Shoes ,Jewelry ,Men ]),In(i_class, [womens ,birdal ,pants ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Home,Books,Electronics]),In(i_class, [wallpaper,parenting,musical])),And(In(i_category, [Shoes,Jewelry,Men]),In(i_class, [womens,birdal,pants]))), IsNotNull(i_item_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -117,7 +117,7 @@ Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] (19) Filter [codegen id : 3] Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] -Condition : (((i_category#17 IN (Home ,Books ,Electronics ) AND i_class#16 IN (wallpaper ,parenting ,musical )) OR (i_category#17 IN (Shoes ,Jewelry ,Men ) AND i_class#16 IN (womens ,birdal ,pants ))) AND isnotnull(i_item_sk#14)) +Condition : (((i_category#17 IN (Home,Books,Electronics) AND i_class#16 IN (wallpaper,parenting,musical)) OR (i_category#17 IN (Shoes,Jewelry,Men) AND i_class#16 IN (womens,birdal,pants))) AND isnotnull(i_item_sk#14)) (20) BroadcastExchange Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#17, i_brand#15, s_store_ (29) Filter [codegen id : 7] Input [9]: [i_category#17, i_class#16, i_brand#15, s_store_name#11, s_company_name#12, d_moy#8, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (30) Project [codegen id : 7] Output [8]: [i_category#17, i_class#16, i_brand#15, s_store_name#11, s_company_name#12, d_moy#8, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt index 2ec1ea034361a..0210877799a98 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Home ,Books ,Electronics ]),In(i_class, [wallpaper ,parenting ,musical ])),And(In(i_category, [Shoes ,Jewelry ,Men ]),In(i_class, [womens ,birdal ,pants ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Home,Books,Electronics]),In(i_class, [wallpaper,parenting,musical])),And(In(i_category, [Shoes,Jewelry,Men]),In(i_class, [womens,birdal,pants]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 4] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Home ,Books ,Electronics ) AND i_class#3 IN (wallpaper ,parenting ,musical )) OR (i_category#4 IN (Shoes ,Jewelry ,Men ) AND i_class#3 IN (womens ,birdal ,pants ))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Home,Books,Electronics) AND i_class#3 IN (wallpaper,parenting,musical)) OR (i_category#4 IN (Shoes,Jewelry,Men) AND i_class#3 IN (womens,birdal,pants))) AND isnotnull(i_item_sk#1)) (4) Scan parquet default.store_sales Output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt index 9d369b5d211fc..09e8dbc57e336 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt @@ -57,7 +57,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((((isnotnull(d_date#6) AND (d_date#6 >= 2001-01-01)) AND (d_date#6 <= 2001-01-31)) AND (d_date_sk#5 >= 2451911)) AND (d_date_sk#5 <= 2451941)) AND isnotnull(d_date_sk#5)) +Condition : (((((isnotnull(d_date#6) AND (d_date#6 >= 11323)) AND (d_date#6 <= 11353)) AND (d_date_sk#5 >= 2451911)) AND (d_date_sk#5 <= 2451941)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Jewelry ,Sports ,Books ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Jewelry,Sports,Books]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -96,7 +96,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Jewelry ,Sports ,Books ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Jewelry,Sports,Books) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt index e31c029b4932e..d46901132fcc3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt @@ -46,7 +46,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Jewelry ,Sports ,Books ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Jewelry,Sports,Books]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -54,7 +54,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Jewelry ,Sports ,Books ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Jewelry,Sports,Books) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((((isnotnull(d_date#13) AND (d_date#13 >= 2001-01-01)) AND (d_date#13 <= 2001-01-31)) AND (d_date_sk#12 >= 2451911)) AND (d_date_sk#12 <= 2451941)) AND isnotnull(d_date_sk#12)) +Condition : (((((isnotnull(d_date#13) AND (d_date#13 >= 11323)) AND (d_date#13 <= 11353)) AND (d_date_sk#12 >= 2451911)) AND (d_date_sk#12 <= 2451941)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt index 039a38acb9f22..4e258bb08b1d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 0c22f6333e5a9..0ae1f849bca42 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(ws_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt index 55dca59270306..e400eea2cea66 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt @@ -58,7 +58,7 @@ Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnot Output [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -66,7 +66,7 @@ Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] (6) Filter [codegen id : 1] Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] -Condition : (isnotnull(cd_demo_sk#12) AND ((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) OR ((cd_marital_status#13 = S) AND (cd_education_status#14 = College ))) OR ((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )))) +Condition : (isnotnull(cd_demo_sk#12) AND ((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) OR ((cd_marital_status#13 = S) AND (cd_education_status#14 = College))) OR ((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)))) (7) BroadcastExchange Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] @@ -75,7 +75,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (8) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#12] -Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) (9) Project [codegen id : 6] Output [11]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#13, cd_education_status#14] @@ -102,7 +102,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#16] -Join condition: (((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#17 = 3)) OR (((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#17 = 1))) OR (((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#17 = 1))) +Join condition: (((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#17 = 3)) OR (((((cd_marital_status#13 = S) AND (cd_education_status#14 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#17 = 1))) OR (((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#17 = 1))) (15) Project [codegen id : 6] Output [7]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index 027fde38ff7fa..560738591c930 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -147,7 +147,7 @@ Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ex Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct (25) ColumnarToRow [codegen id : 4] @@ -155,7 +155,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (26) Filter [codegen id : 4] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : (isnotnull(cd_demo_sk#21) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College ))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )))) +Condition : (isnotnull(cd_demo_sk#21) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)))) (27) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] @@ -164,7 +164,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (28) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#21] -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) (29) Project [codegen id : 6] Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] @@ -191,7 +191,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#25] -Join condition: (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#26 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#26 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#26 = 1))) +Join condition: (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#26 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#26 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#26 = 1))) (35) Project [codegen id : 6] Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 9c56438870f64..adfa3e44f996d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -203,7 +203,7 @@ Input [2]: [d_date_sk#22, d_date#23] (36) Filter [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2002-02-01)) AND (d_date#23 <= 2002-04-02)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11719)) AND (d_date#23 <= 11779)) AND isnotnull(d_date_sk#22)) (37) Project [codegen id : 10] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index dde6a9f564859..5640564564396 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -141,7 +141,7 @@ Input [2]: [d_date_sk#16, d_date#17] (22) Filter [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2002-02-01)) AND (d_date#17 <= 2002-04-02)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11719)) AND (d_date#17 <= 11779)) AND isnotnull(d_date_sk#16)) (23) Project [codegen id : 8] Output [1]: [d_date_sk#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index 8918c2a36e2ec..97467fe1dc1db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -77,7 +77,7 @@ Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isno Output [2]: [d_date_sk#8, d_quarter_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -85,7 +85,7 @@ Input [2]: [d_date_sk#8, d_quarter_name#9] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#8, d_quarter_name#9] -Condition : ((isnotnull(d_quarter_name#9) AND (rpad(d_quarter_name#9, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#8)) +Condition : ((isnotnull(d_quarter_name#9) AND (d_quarter_name#9 = 2001Q1)) AND isnotnull(d_date_sk#8)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#8] @@ -197,7 +197,7 @@ Condition : ((isnotnull(sr_customer_sk#21) AND isnotnull(sr_item_sk#20)) AND isn Output [2]: [d_date_sk#26, d_quarter_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] @@ -205,7 +205,7 @@ Input [2]: [d_date_sk#26, d_quarter_name#27] (33) Filter [codegen id : 9] Input [2]: [d_date_sk#26, d_quarter_name#27] -Condition : (rpad(d_quarter_name#27, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) +Condition : (d_quarter_name#27 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) (34) Project [codegen id : 9] Output [1]: [d_date_sk#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index dafa24943be82..32c8de9a1d16e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -124,7 +124,7 @@ Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_i Output [2]: [d_date_sk#21, d_quarter_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 3] @@ -132,7 +132,7 @@ Input [2]: [d_date_sk#21, d_quarter_name#22] (18) Filter [codegen id : 3] Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : ((isnotnull(d_quarter_name#22) AND (rpad(d_quarter_name#22, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#21)) +Condition : ((isnotnull(d_quarter_name#22) AND (d_quarter_name#22 = 2001Q1)) AND isnotnull(d_date_sk#21)) (19) Project [codegen id : 3] Output [1]: [d_date_sk#21] @@ -155,7 +155,7 @@ Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_re Output [2]: [d_date_sk#24, d_quarter_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] @@ -163,7 +163,7 @@ Input [2]: [d_date_sk#24, d_quarter_name#25] (25) Filter [codegen id : 4] Input [2]: [d_date_sk#24, d_quarter_name#25] -Condition : (rpad(d_quarter_name#25, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) +Condition : (d_quarter_name#25 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) (26) Project [codegen id : 4] Output [1]: [d_date_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt index 85aed12cc9d4f..020e92edb9d38 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt @@ -73,7 +73,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1) Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14 (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown)) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] Output [2]: [cd_demo_sk#11, cd_dep_count#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index ed68f03734b13..b3b8cabfb6880 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -67,7 +67,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1) Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -75,7 +75,7 @@ Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14 (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown)) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] Output [2]: [cd_demo_sk#11, cd_dep_count#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt index a40286f9f8f83..adb5346969d06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index 4f5eebc0c300c..1816433bdd129 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt index efb45b5ccdb7f..a04d4e6f5a5c2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt @@ -148,7 +148,7 @@ Results [4]: [w_warehouse_name#14, i_item_id#7, sum(CASE WHEN (d_date#11 < 2000- (26) Filter [codegen id : 5] Input [4]: [w_warehouse_name#14, i_item_id#7, inv_before#23, inv_after#24] -Condition : ((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (((inv_before#23 > 0) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5))) +Condition : (((((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5)) (27) TakeOrderedAndProject Input [4]: [w_warehouse_name#14, i_item_id#7, inv_before#23, inv_after#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 7fcbe1befa6b4..38d49852134cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -148,7 +148,7 @@ Results [4]: [w_warehouse_name#7, i_item_id#10, sum(CASE WHEN (d_date#14 < 2000- (26) Filter [codegen id : 5] Input [4]: [w_warehouse_name#7, i_item_id#10, inv_before#23, inv_after#24] -Condition : ((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (((inv_before#23 > 0) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5))) +Condition : (((((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5)) (27) TakeOrderedAndProject Input [4]: [w_warehouse_name#7, i_item_id#10, inv_before#23, inv_after#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index 431cf3ef92deb..e9a0ada3801b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -73,7 +73,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_ (7) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Condition : ((isnotnull(i_color#10) AND (i_color#10 = pale )) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_color#10) AND (i_color#10 = pale)) AND isnotnull(i_item_sk#7)) (8) BroadcastExchange Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index f73a5a5e052e3..3b304ebe3f649 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -144,7 +144,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -152,7 +152,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] -Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale )) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale)) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 163a4e7e3ddad..695f6efe560e4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -73,7 +73,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_ (7) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Condition : ((isnotnull(i_color#10) AND (i_color#10 = chiffon )) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_color#10) AND (i_color#10 = chiffon)) AND isnotnull(i_item_sk#7)) (8) BroadcastExchange Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index 09830b1be656e..49f76525f88d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -144,7 +144,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -152,7 +152,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] -Condition : ((isnotnull(i_color#21) AND (i_color#21 = chiffon )) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = chiffon)) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt index 40565dc76a7ba..e38776ac2784b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isn Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index c86f121ad35a9..0de1f44569c19 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isn Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt index 8c530f5a37885..f4a07e6ac8f21 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnu Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index 8c530f5a37885..f4a07e6ac8f21 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnu Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index 01d0ca4a71e94..b70c36db4bc9d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 70575554e273e..9537689459170 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -105,7 +105,7 @@ Input [2]: [d_date_sk#8, d_date#9] (16) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) (17) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 1349aa54f3cc2..cba099c2bb5b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -163,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 1349aa54f3cc2..cba099c2bb5b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -163,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt index ea9994910c3b9..60b7965371e06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index df5b93da51771..6979de4765247 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt index 1f2c169ba0921..3f5ed7b48a9db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt @@ -95,7 +95,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10988)) AND (d_date#12 <= 11048)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt index 9cc89345f5a1c..04c3f07b903f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt @@ -92,7 +92,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10988)) AND (d_date#12 <= 11048)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt index 71accc7295b0f..776585e5bd97f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt @@ -177,11 +177,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (23) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] (24) Sort [codegen id : 8] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (25) Scan parquet default.catalog_sales Output [2]: [cs_bill_customer_sk#15, cs_sold_date_sk#16] @@ -254,15 +254,15 @@ Results [3]: [c_last_name#22, c_first_name#21, d_date#18] (40) Exchange Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] (41) Sort [codegen id : 16] Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 1970-01-01) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 0) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 (42) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18)] Join condition: None (43) HashAggregate [codegen id : 17] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (46) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] (47) Sort [codegen id : 19] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (48) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#27, ws_sold_date_sk#28] @@ -362,15 +362,15 @@ Results [3]: [c_last_name#34, c_first_name#33, d_date#30] (63) Exchange Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] +Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] (64) Sort [codegen id : 27] Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 1970-01-01) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 0) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 (65) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30)] Join condition: None (66) HashAggregate [codegen id : 28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index c4e679eb902e0..cedc6c7f4c576 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -205,11 +205,11 @@ Results [3]: [c_last_name#19, c_first_name#18, d_date#16] (32) BroadcastExchange Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#21] (33) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 0), isnull(d_date#16)] Join condition: None (34) HashAggregate [codegen id : 12] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#28, c_first_name#27, d_date#25] (48) BroadcastExchange Input [3]: [c_last_name#28, c_first_name#27, d_date#25] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#30] (49) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 1970-01-01), isnull(d_date#25)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 0), isnull(d_date#25)] Join condition: None (50) HashAggregate [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt index 68ab7a8bbecab..a53bec582637d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) +Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -280,7 +280,7 @@ Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_qu (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stdev#53, mean#54] -Condition : ((isnotnull(mean#54) AND isnotnull(stdev#53)) AND (NOT (mean#54 = 0.0) AND ((stdev#53 / mean#54) > 1.0))) +Condition : (((isnotnull(mean#54) AND isnotnull(stdev#53)) AND NOT (mean#54 = 0.0)) AND ((stdev#53 / mean#54) > 1.0)) (50) Project [codegen id : 11] Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, CASE WHEN (mean#54 = 0.0) THEN null ELSE (stdev#53 / mean#54) END AS cov#55] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index 7abd3bb1acb11..c61480a05a4cf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -173,7 +173,7 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) +Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -269,7 +269,7 @@ Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_qu (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] -Condition : ((isnotnull(mean#53) AND isnotnull(stdev#52)) AND (NOT (mean#53 = 0.0) AND ((stdev#52 / mean#53) > 1.0))) +Condition : (((isnotnull(mean#53) AND isnotnull(stdev#52)) AND NOT (mean#53 = 0.0)) AND ((stdev#52 / mean#53) > 1.0)) (48) Project [codegen id : 9] Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt index f0dfbac4f3a0f..6bc5980231252 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) +Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -280,7 +280,7 @@ Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_qu (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stdev#53, mean#54] -Condition : ((isnotnull(mean#54) AND isnotnull(stdev#53)) AND (NOT (mean#54 = 0.0) AND ((stdev#53 / mean#54) > 1.0))) +Condition : (((isnotnull(mean#54) AND isnotnull(stdev#53)) AND NOT (mean#54 = 0.0)) AND ((stdev#53 / mean#54) > 1.0)) (50) Project [codegen id : 11] Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, CASE WHEN (mean#54 = 0.0) THEN null ELSE (stdev#53 / mean#54) END AS cov#55] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index d1a5ecaa6a4d8..0ffe40240be2b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -173,7 +173,7 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) +Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -269,7 +269,7 @@ Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_qu (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] -Condition : ((isnotnull(mean#53) AND isnotnull(stdev#52)) AND (NOT (mean#53 = 0.0) AND ((stdev#52 / mean#53) > 1.0))) +Condition : (((isnotnull(mean#53) AND isnotnull(stdev#52)) AND NOT (mean#53 = 0.0)) AND ((stdev#52 / mean#53) > 1.0)) (48) Project [codegen id : 9] Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index abb121a4c17ea..13d73e61e1443 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -66,7 +66,7 @@ Results [2]: [i_manufact#2, count#9] (10) Exchange Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), ENSURE_REQUIREMENTS, [id=#10] +Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] (11) HashAggregate [codegen id : 2] Input [2]: [i_manufact#2, count#9] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index abb121a4c17ea..13d73e61e1443 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -66,7 +66,7 @@ Results [2]: [i_manufact#2, count#9] (10) Exchange Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), ENSURE_REQUIREMENTS, [id=#10] +Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] (11) HashAggregate [codegen id : 2] Input [2]: [i_manufact#2, count#9] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index fdb95bd5de9d3..35924d2ffd9a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -95,7 +95,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 10] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) +Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) (14) Project [codegen id : 10] Output [2]: [item_sk#11, rnk#17] @@ -133,7 +133,7 @@ Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, (22) Filter [codegen id : 7] Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) +Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) (23) Project [codegen id : 7] Output [2]: [item_sk#21, rnk#25] @@ -141,7 +141,7 @@ Input [3]: [item_sk#21, rank_col#22, rnk#25] (24) BroadcastExchange Input [2]: [item_sk#21, rnk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#26] (25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index f52ad2e63762e..4ca133ffd838d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -96,7 +96,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 4] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) +Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) (14) Project [codegen id : 4] Output [2]: [item_sk#11, rnk#17] @@ -138,7 +138,7 @@ Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, (23) Filter [codegen id : 8] Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) +Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) (24) Project [codegen id : 8] Output [2]: [item_sk#21, rnk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt index 2f87af685f2e5..bd8d979c32d17 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt @@ -79,7 +79,7 @@ Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_ Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree)),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College)))] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -87,7 +87,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (12) Filter [codegen id : 2] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree ))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College )))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College)))) (13) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -96,7 +96,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) (15) Project [codegen id : 5] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index 2f87af685f2e5..bd8d979c32d17 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt @@ -79,7 +79,7 @@ Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_ Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree)),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College)))] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -87,7 +87,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (12) Filter [codegen id : 2] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree ))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College )))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College)))) (13) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -96,7 +96,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) (15) Project [codegen id : 5] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index a754cef8f2962..2d3dea5d84565 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -160,7 +160,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 2000-08-23)) AND (d_date#26 <= 2000-09-06)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 11192)) AND (d_date#26 <= 11206)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index b3eb1f9592ff3..f6fe8bee89369 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -130,7 +130,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11192)) AND (d_date#23 <= 11206)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt index e0cd0729e0e32..1c18ab278209b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 3bf06577e3212..3c41db90ad1aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index 85906beeff369..c8058eeee7ca1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -188,7 +188,7 @@ Input [4]: [sold_date_sk#12, customer_sk#13, item_sk#14, d_date_sk#21] Output [3]: [i_item_sk#25, i_class#26, i_category#27] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women), EqualTo(i_class,maternity), IsNotNull(i_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 6] @@ -196,7 +196,7 @@ Input [3]: [i_item_sk#25, i_class#26, i_category#27] (29) Filter [codegen id : 6] Input [3]: [i_item_sk#25, i_class#26, i_category#27] -Condition : ((((isnotnull(i_category#27) AND isnotnull(i_class#26)) AND (i_category#27 = Women )) AND (i_class#26 = maternity )) AND isnotnull(i_item_sk#25)) +Condition : ((((isnotnull(i_category#27) AND isnotnull(i_class#26)) AND (i_category#27 = Women)) AND (i_class#26 = maternity)) AND isnotnull(i_item_sk#25)) (30) Project [codegen id : 6] Output [1]: [i_item_sk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 2c1f7c6195ba7..b14ae55d75093 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -109,7 +109,7 @@ Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] Output [3]: [i_item_sk#14, i_class#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women), EqualTo(i_class,maternity), IsNotNull(i_item_sk)] ReadSchema: struct (11) ColumnarToRow [codegen id : 3] @@ -117,7 +117,7 @@ Input [3]: [i_item_sk#14, i_class#15, i_category#16] (12) Filter [codegen id : 3] Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women )) AND (i_class#15 = maternity )) AND isnotnull(i_item_sk#14)) +Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women)) AND (i_class#15 = maternity)) AND isnotnull(i_item_sk#14)) (13) Project [codegen id : 3] Output [1]: [i_item_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index f6dd5a59de4d4..ee3abb39ed053 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -467,7 +467,7 @@ Input [2]: [d_date#5, d_week_seq#6] (82) Filter [codegen id : 1] Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_date#5) AND (d_date#5 = 2000-01-03)) +Condition : (isnotnull(d_date#5) AND (d_date#5 = 10959)) (83) Project [codegen id : 1] Output [1]: [d_week_seq#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index ab4c4ad4ae65b..dd37b8801e957 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -467,7 +467,7 @@ Input [2]: [d_date#8, d_week_seq#9] (82) Filter [codegen id : 1] Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_date#8) AND (d_date#8 = 2000-01-03)) +Condition : (isnotnull(d_date#8) AND (d_date#8 = 10959)) (83) Project [codegen id : 1] Output [1]: [d_week_seq#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt index 2b8dbaddacad8..3533986d86b84 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,refernece ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,refernece,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,refernece ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,refernece,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index efb27d8cd8b11..622b073a4ae10 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,refernece ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,refernece,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,refernece ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,refernece,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt index bd8dc07f42967..5d74065179f44 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt @@ -741,7 +741,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (121) ColumnarToRow [codegen id : 40] @@ -749,7 +749,7 @@ Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] (122) Filter [codegen id : 40] Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] -Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) +Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) (123) Project [codegen id : 40] Output [2]: [i_item_sk#93, i_product_name#96] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index 216e7f5530128..9cc3415e6e146 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -651,7 +651,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 24] @@ -659,7 +659,7 @@ Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] (106) Filter [codegen id : 24] Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] -Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) +Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) (107) Project [codegen id : 24] Output [2]: [i_item_sk#87, i_product_name#90] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt index 3a19c3923737d..075fd1864222c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt @@ -183,7 +183,7 @@ Arguments: [rank(sumsales#38) windowspecdefinition(i_category#23, sumsales#38 DE (33) Filter [codegen id : 10] Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#38, rk#40] -Condition : (rk#40 <= 100) +Condition : (isnotnull(rk#40) AND (rk#40 <= 100)) (34) TakeOrderedAndProject Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#38, rk#40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index 3d42f49415640..ba71667cfebe5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -168,7 +168,7 @@ Arguments: [rank(sumsales#37) windowspecdefinition(i_category#22, sumsales#37 DE (30) Filter [codegen id : 7] Input [10]: [i_category#22, i_class#23, i_brand#24, i_product_name#25, d_year#26, d_qoy#27, d_moy#28, s_store_id#29, sumsales#37, rk#39] -Condition : (rk#39 <= 100) +Condition : (isnotnull(rk#39) AND (rk#39 <= 100)) (31) TakeOrderedAndProject Input [10]: [i_category#22, i_class#23, i_brand#24, i_product_name#25, d_year#26, d_qoy#27, d_moy#28, s_store_id#29, sumsales#37, rk#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt index b138d059eaecb..ab2cc784a5f22 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index 5f6002b84f411..9c914d56f46a2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index 9e2d33f2bca19..d31dbc3498ead 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index cd12e362c44f0..26fec145f4211 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt index cd88a84ede266..3daa8b66851f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt @@ -180,7 +180,7 @@ Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] @@ -188,7 +188,7 @@ Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (33) Filter [codegen id : 8] Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) +Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) (34) Project [codegen id : 8] Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index cd88a84ede266..3daa8b66851f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -180,7 +180,7 @@ Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] @@ -188,7 +188,7 @@ Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (33) Filter [codegen id : 8] Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) +Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) (34) Project [codegen id : 8] Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index 339dc5d95e4c4..04172ab353f25 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -100,7 +100,7 @@ Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotn Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -108,7 +108,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000 )) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000)) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index 177b8e681608a..fe49873e2e3f3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -207,7 +207,7 @@ Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_ Output [2]: [hd_demo_sk#24, hd_buy_potential#25] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -215,7 +215,7 @@ Input [2]: [hd_demo_sk#24, hd_buy_potential#25] (31) Filter [codegen id : 5] Input [2]: [hd_demo_sk#24, hd_buy_potential#25] -Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = >10000 )) AND isnotnull(hd_demo_sk#24)) +Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = >10000)) AND isnotnull(hd_demo_sk#24)) (32) Project [codegen id : 5] Output [1]: [hd_demo_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt index aec6d66c98fdd..e658e11ade4a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index 646a8fbc11a3a..b45f0513ba576 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt index f87f822166e23..9472a8935eb8b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index f87f822166e23..9472a8935eb8b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt index e5c59a0ecfaad..6e86260f0aba8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt @@ -244,7 +244,7 @@ Input [2]: [d_date_sk#22, d_date#23] (30) Filter [codegen id : 7] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-22)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11192)) AND (d_date#23 <= 11222)) AND isnotnull(d_date_sk#22)) (31) Project [codegen id : 7] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index 3d087efe472bb..87abdc02b21a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -182,7 +182,7 @@ Input [2]: [d_date_sk#16, d_date#17] (16) Filter [codegen id : 5] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-08-23)) AND (d_date#17 <= 2000-09-22)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11192)) AND (d_date#17 <= 11222)) AND isnotnull(d_date_sk#16)) (17) Project [codegen id : 5] Output [1]: [d_date_sk#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt index 6725e273a3acc..8b09336f86535 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt @@ -95,7 +95,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 11102)) AND (d_date#12 <= 11162)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt index a03333f7623cc..340fcf5fa31fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt @@ -92,7 +92,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 11102)) AND (d_date#12 <= 11162)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt index 7f7c9ac72707e..0073fd978d748 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt @@ -245,7 +245,7 @@ Arguments: [wr_refunded_cdemo_sk#13 ASC NULLS FIRST, wr_returning_cdemo_sk#15 AS Output [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct (43) ColumnarToRow [codegen id : 11] @@ -253,7 +253,7 @@ Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] (44) Filter [codegen id : 11] Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] -Condition : (((isnotnull(cd_demo_sk#33) AND isnotnull(cd_marital_status#34)) AND isnotnull(cd_education_status#35)) AND ((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree )) OR ((cd_marital_status#34 = S) AND (cd_education_status#35 = College ))) OR ((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree )))) +Condition : (((isnotnull(cd_demo_sk#33) AND isnotnull(cd_marital_status#34)) AND isnotnull(cd_education_status#35)) AND ((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree)) OR ((cd_marital_status#34 = S) AND (cd_education_status#35 = College))) OR ((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree)))) (45) BroadcastExchange Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] @@ -293,7 +293,7 @@ Arguments: [cast(cd_demo_sk#33 as bigint) ASC NULLS FIRST, cast(cd_demo_sk#37 as (53) SortMergeJoin [codegen id : 14] Left keys [2]: [wr_refunded_cdemo_sk#13, wr_returning_cdemo_sk#15] Right keys [2]: [cast(cd_demo_sk#33 as bigint), cast(cd_demo_sk#37 as bigint)] -Join condition: ((((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#34 = S) AND (cd_education_status#35 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree)) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#34 = S) AND (cd_education_status#35 = College)) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree)) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) (54) Project [codegen id : 14] Output [4]: [ws_quantity#4, wr_fee#18, wr_refunded_cash#19, r_reason_desc#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index 74a9aa3dec985..c528cf8f91a55 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -130,7 +130,7 @@ Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -138,7 +138,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (19) Filter [codegen id : 3] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : (((isnotnull(cd_demo_sk#21) AND isnotnull(cd_marital_status#22)) AND isnotnull(cd_education_status#23)) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College ))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )))) +Condition : (((isnotnull(cd_demo_sk#21) AND isnotnull(cd_marital_status#22)) AND isnotnull(cd_education_status#23)) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)))) (20) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] @@ -147,7 +147,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [wr_refunded_cdemo_sk#11] Right keys [1]: [cast(cd_demo_sk#21 as bigint)] -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) (22) Project [codegen id : 8] Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, cd_marital_status#22, cd_education_status#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt index 92895cb566fd2..888fd8e520796 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt @@ -177,11 +177,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (23) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] (24) Sort [codegen id : 8] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (25) Scan parquet default.catalog_sales Output [2]: [cs_bill_customer_sk#15, cs_sold_date_sk#16] @@ -254,15 +254,15 @@ Results [3]: [c_last_name#22, c_first_name#21, d_date#18] (40) Exchange Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] (41) Sort [codegen id : 16] Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 1970-01-01) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 0) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 (42) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18)] Join condition: None (43) HashAggregate [codegen id : 17] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (46) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] (47) Sort [codegen id : 19] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (48) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#27, ws_sold_date_sk#28] @@ -362,15 +362,15 @@ Results [3]: [c_last_name#34, c_first_name#33, d_date#30] (63) Exchange Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] +Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] (64) Sort [codegen id : 27] Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 1970-01-01) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 0) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 (65) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30)] Join condition: None (66) HashAggregate [codegen id : 28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index 27e16b75638a8..bc62b67957935 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -205,11 +205,11 @@ Results [3]: [c_last_name#19, c_first_name#18, d_date#16] (32) BroadcastExchange Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#21] (33) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 0), isnull(d_date#16)] Join condition: None (34) HashAggregate [codegen id : 12] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#28, c_first_name#27, d_date#25] (48) BroadcastExchange Input [3]: [c_last_name#28, c_first_name#27, d_date#25] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#30] (49) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 1970-01-01), isnull(d_date#25)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 0), isnull(d_date#25)] Join condition: None (50) HashAggregate [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt index b3d0c07cd80fe..c9ebf0e9c9426 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Books ,Electronics ,Sports ]),In(i_class, [computers ,stereo ,football ])),And(In(i_category, [Men ,Jewelry ,Women ]),In(i_class, [shirts ,birdal ,dresses ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Books,Electronics,Sports]),In(i_class, [computers,stereo,football])),And(In(i_category, [Men,Jewelry,Women]),In(i_class, [shirts,birdal,dresses]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 1] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Books ,Electronics ,Sports ) AND i_class#3 IN (computers ,stereo ,football )) OR (i_category#4 IN (Men ,Jewelry ,Women ) AND i_class#3 IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Books,Electronics,Sports) AND i_class#3 IN (computers,stereo,football)) OR (i_category#4 IN (Men,Jewelry,Women) AND i_class#3 IN (shirts,birdal,dresses))) AND isnotnull(i_item_sk#1)) (4) BroadcastExchange Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index f61c8e6945003..cbf6273e28568 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Books ,Electronics ,Sports ]),In(i_class, [computers ,stereo ,football ])),And(In(i_category, [Men ,Jewelry ,Women ]),In(i_class, [shirts ,birdal ,dresses ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Books,Electronics,Sports]),In(i_class, [computers,stereo,football])),And(In(i_category, [Men,Jewelry,Women]),In(i_class, [shirts,birdal,dresses]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 4] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Books ,Electronics ,Sports ) AND i_class#3 IN (computers ,stereo ,football )) OR (i_category#4 IN (Men ,Jewelry ,Women ) AND i_class#3 IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Books,Electronics,Sports) AND i_class#3 IN (computers,stereo,football)) OR (i_category#4 IN (Men,Jewelry,Women) AND i_class#3 IN (shirts,birdal,dresses))) AND isnotnull(i_item_sk#1)) (4) Scan parquet default.store_sales Output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt index 3554ce5dcb117..e38c455eb6be2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt @@ -187,7 +187,7 @@ Input [5]: [cr_call_center_sk#2, cr_net_loss#3, c_current_cdemo_sk#12, c_current Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown )),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree ))), IsNotNull(cd_demo_sk)] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 5] @@ -195,7 +195,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (33) Filter [codegen id : 5] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Unknown )) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = Advanced Degree ))) AND isnotnull(cd_demo_sk#21)) +Condition : ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Unknown)) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = Advanced Degree))) AND isnotnull(cd_demo_sk#21)) (34) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index fc0da4adf7b16..c14a8421bc3b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -183,7 +183,7 @@ Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_curre Output [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown )),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree ))), IsNotNull(cd_demo_sk)] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 5] @@ -191,7 +191,7 @@ Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] (32) Filter [codegen id : 5] Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Condition : ((((cd_marital_status#24 = M) AND (cd_education_status#25 = Unknown )) OR ((cd_marital_status#24 = W) AND (cd_education_status#25 = Advanced Degree ))) AND isnotnull(cd_demo_sk#23)) +Condition : ((((cd_marital_status#24 = M) AND (cd_education_status#25 = Unknown)) OR ((cd_marital_status#24 = W) AND (cd_education_status#25 = Advanced Degree))) AND isnotnull(cd_demo_sk#23)) (33) BroadcastExchange Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index 1fdf74c2ce164..8297114e1e9ab 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -84,7 +84,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index 06c28e176aa1a..faf82026138f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -108,7 +108,7 @@ Input [2]: [d_date_sk#8, d_date#9] (16) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) (17) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index 2466a4d38f21e..aec81c0d5c7aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -47,7 +47,7 @@ Input [5]: [sr_item_sk#1, sr_reason_sk#2, sr_ticket_number#3, sr_return_quantity Output [2]: [r_reason_sk#6, r_reason_desc#7] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -55,7 +55,7 @@ Input [2]: [r_reason_sk#6, r_reason_desc#7] (7) Filter [codegen id : 1] Input [2]: [r_reason_sk#6, r_reason_desc#7] -Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28 )) AND isnotnull(r_reason_sk#6)) +Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28)) AND isnotnull(r_reason_sk#6)) (8) Project [codegen id : 1] Output [1]: [r_reason_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 1f3470198cd20..1875c5418a890 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -85,7 +85,7 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, s Output [2]: [r_reason_sk#14, r_reason_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 5] @@ -93,7 +93,7 @@ Input [2]: [r_reason_sk#14, r_reason_desc#15] (16) Filter [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] -Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28 )) AND isnotnull(r_reason_sk#14)) +Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28)) AND isnotnull(r_reason_sk#14)) (17) Project [codegen id : 5] Output [1]: [r_reason_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index c77590bf71044..bee7110ecd6dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -175,7 +175,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Output [2]: [web_site_sk#20, web_company_name#21] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 10] @@ -183,7 +183,7 @@ Input [2]: [web_site_sk#20, web_company_name#21] (31) Filter [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] -Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri )) AND isnotnull(web_site_sk#20)) +Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri)) AND isnotnull(web_site_sk#20)) (32) Project [codegen id : 10] Output [1]: [web_site_sk#20] @@ -214,7 +214,7 @@ Input [2]: [d_date_sk#23, d_date#24] (38) Filter [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] -Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 1999-02-01)) AND (d_date#24 <= 1999-04-02)) AND isnotnull(d_date_sk#23)) +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 10623)) AND (d_date#24 <= 10683)) AND isnotnull(d_date_sk#23)) (39) Project [codegen id : 11] Output [1]: [d_date_sk#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 9558a01423452..efa09c2f625cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -152,7 +152,7 @@ Input [2]: [d_date_sk#17, d_date#18] (24) Filter [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-01)) AND (d_date#18 <= 1999-04-02)) AND isnotnull(d_date_sk#17)) +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10623)) AND (d_date#18 <= 10683)) AND isnotnull(d_date_sk#17)) (25) Project [codegen id : 9] Output [1]: [d_date_sk#17] @@ -206,7 +206,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_ Output [2]: [web_site_sk#23, web_company_name#24] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] @@ -214,7 +214,7 @@ Input [2]: [web_site_sk#23, web_company_name#24] (38) Filter [codegen id : 11] Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) +Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri)) AND isnotnull(web_site_sk#23)) (39) Project [codegen id : 11] Output [1]: [web_site_sk#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index 320a93e19bb27..725939ef24f1b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -252,7 +252,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Output [2]: [web_site_sk#24, web_company_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 19] @@ -260,7 +260,7 @@ Input [2]: [web_site_sk#24, web_company_name#25] (46) Filter [codegen id : 19] Input [2]: [web_site_sk#24, web_company_name#25] -Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri )) AND isnotnull(web_site_sk#24)) +Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri)) AND isnotnull(web_site_sk#24)) (47) Project [codegen id : 19] Output [1]: [web_site_sk#24] @@ -291,7 +291,7 @@ Input [2]: [d_date_sk#27, d_date#28] (53) Filter [codegen id : 20] Input [2]: [d_date_sk#27, d_date#28] -Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 1999-02-01)) AND (d_date#28 <= 1999-04-02)) AND isnotnull(d_date_sk#27)) +Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 10623)) AND (d_date#28 <= 10683)) AND isnotnull(d_date_sk#27)) (54) Project [codegen id : 20] Output [1]: [d_date_sk#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 2f719a4be1630..9f15375b5cfc0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -229,7 +229,7 @@ Input [2]: [d_date_sk#21, d_date#22] (39) Filter [codegen id : 18] Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-01)) AND (d_date#22 <= 1999-04-02)) AND isnotnull(d_date_sk#21)) +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 10623)) AND (d_date#22 <= 10683)) AND isnotnull(d_date_sk#21)) (40) Project [codegen id : 18] Output [1]: [d_date_sk#21] @@ -283,7 +283,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_ Output [2]: [web_site_sk#27, web_company_name#28] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 20] @@ -291,7 +291,7 @@ Input [2]: [web_site_sk#27, web_company_name#28] (53) Filter [codegen id : 20] Input [2]: [web_site_sk#27, web_company_name#28] -Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) +Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri)) AND isnotnull(web_site_sk#27)) (54) Project [codegen id : 20] Output [1]: [web_site_sk#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt index 98c1bc5671cd9..7b8e4ff5516a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt @@ -57,7 +57,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -96,7 +96,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 1507b5ccbc0ae..3457f398b7dea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -46,7 +46,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -54,7 +54,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt index dd85594160c8f..15c445b6b1325 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index f6dd412f5c4b1..774081bc5c3f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(ws_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt index a74ad3e4b8b77..8bc09f9457fe7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index c6a345be29c9d..49b35aacfcf2f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt index 8031170978906..d4d0015a9fec0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt @@ -157,7 +157,7 @@ Input [12]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk# Output [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 4] @@ -165,7 +165,7 @@ Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, (26) Filter [codegen id : 4] Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Condition : ((isnotnull(i_color#27) AND (i_color#27 = pale )) AND isnotnull(i_item_sk#24)) +Condition : ((isnotnull(i_color#27) AND (i_color#27 = pale)) AND isnotnull(i_item_sk#24)) (27) BroadcastExchange Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index 540feb0d88a3f..6835ff84f5bc7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -146,7 +146,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -154,7 +154,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] -Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale )) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale)) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt index f6151f4d79ec1..104340f2fd6db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt index e4d14f842f5a9..f106fba199355 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 5374a708a0295..e04148fad545e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -173,7 +173,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 1998-08-04)) AND (d_date#26 <= 1998-08-18)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 10442)) AND (d_date#26 <= 10456)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 89362fb4f0efe..36b1ff63b2065 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -143,7 +143,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 10442)) AND (d_date#23 <= 10456)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt index 2a9e1ad7c715b..c096a28386361 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt @@ -741,7 +741,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (121) ColumnarToRow [codegen id : 40] @@ -749,7 +749,7 @@ Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] (122) Filter [codegen id : 40] Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] -Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) +Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) (123) Project [codegen id : 40] Output [2]: [i_item_sk#93, i_product_name#96] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 5c2ac1abaaa7e..9424b7f1e2225 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -651,7 +651,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 24] @@ -659,7 +659,7 @@ Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] (106) Filter [codegen id : 24] Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] -Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) +Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) (107) Project [codegen id : 24] Output [2]: [i_item_sk#87, i_product_name#90] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt index 413a918da6a0f..6d9adf1d38106 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt @@ -445,7 +445,7 @@ Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DE (73) Filter [codegen id : 82] Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] -Condition : (rk#148 <= 100) +Condition : (isnotnull(rk#148) AND (rk#148 <= 100)) (74) TakeOrderedAndProject Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index fa822f3ac9ed7..ae6b3ff4d7542 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -430,7 +430,7 @@ Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DE (70) Filter [codegen id : 55] Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] -Condition : (rk#147 <= 100) +Condition : (isnotnull(rk#147) AND (rk#147 <= 100)) (71) TakeOrderedAndProject Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt index e024d06c710a7..3c65529504320 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 5ae5fd82839cc..406acb0e0a27f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index fe8ceb415f571..0d90342fc9bbf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -100,7 +100,7 @@ Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotn Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -108,7 +108,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000 )) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000)) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index 0e67565e856ad..c897c78880bc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -207,7 +207,7 @@ Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_ Output [2]: [hd_demo_sk#24, hd_buy_potential#25] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -215,7 +215,7 @@ Input [2]: [hd_demo_sk#24, hd_buy_potential#25] (31) Filter [codegen id : 5] Input [2]: [hd_demo_sk#24, hd_buy_potential#25] -Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = 1001-5000 )) AND isnotnull(hd_demo_sk#24)) +Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = 1001-5000)) AND isnotnull(hd_demo_sk#24)) (32) Project [codegen id : 5] Output [1]: [hd_demo_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt index 7ecbe2e51c9b4..955dde96eb177 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 7ecbe2e51c9b4..955dde96eb177 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt index 8a384ec4b1795..e192ab8d637de 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt @@ -257,7 +257,7 @@ Input [2]: [d_date_sk#22, d_date#23] (30) Filter [codegen id : 7] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-09-03)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 10442)) AND (d_date#23 <= 10472)) AND isnotnull(d_date_sk#22)) (31) Project [codegen id : 7] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index 596f9497b5905..e027ef7e53e8a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt @@ -195,7 +195,7 @@ Input [2]: [d_date_sk#16, d_date#17] (16) Filter [codegen id : 5] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 1998-08-04)) AND (d_date#17 <= 1998-09-03)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 10442)) AND (d_date#17 <= 10472)) AND isnotnull(d_date_sk#16)) (17) Project [codegen id : 5] Output [1]: [d_date_sk#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt index 89bb3e5d551a8..b036ff059dbb2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt @@ -56,7 +56,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -87,7 +87,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -95,7 +95,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index b7c5f4081a1f3..920a4c2a55a26 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -45,7 +45,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -53,7 +53,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -80,7 +80,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index 518597f59690d..76204c504c0ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -44,6 +44,9 @@ import org.apache.spark.sql.internal.SQLConf * actual explain plan: /path/to/tmp/q1.actual.explain.txt * [actual simplified plan] * + * The explain files are saved to help debug later, they are not checked. Only the simplified + * plans are checked (by string comparison). + * * * To run the entire test suite: * {{{ @@ -98,13 +101,10 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite { new File(goldenFilePath, name) } - private def isApproved( - dir: File, actualSimplifiedPlan: String, actualExplain: String): Boolean = { - val simplifiedFile = new File(dir, "simplified.txt") - val expectedSimplified = FileUtils.readFileToString(simplifiedFile, StandardCharsets.UTF_8) - lazy val explainFile = new File(dir, "explain.txt") - lazy val expectedExplain = FileUtils.readFileToString(explainFile, StandardCharsets.UTF_8) - expectedSimplified == actualSimplifiedPlan && expectedExplain == actualExplain + private def isApproved(dir: File, actualSimplifiedPlan: String): Boolean = { + val file = new File(dir, "simplified.txt") + val expected = FileUtils.readFileToString(file, StandardCharsets.UTF_8) + expected == actualSimplifiedPlan } /** @@ -119,7 +119,7 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite { private def generateGoldenFile(plan: SparkPlan, name: String, explain: String): Unit = { val dir = getDirForTest(name) val simplified = getSimplifiedPlan(plan) - val foundMatch = dir.exists() && isApproved(dir, simplified, explain) + val foundMatch = dir.exists() && isApproved(dir, simplified) if (!foundMatch) { FileUtils.deleteDirectory(dir) @@ -137,7 +137,7 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite { val dir = getDirForTest(name) val tempDir = FileUtils.getTempDirectory val actualSimplified = getSimplifiedPlan(plan) - val foundMatch = isApproved(dir, actualSimplified, explain) + val foundMatch = isApproved(dir, actualSimplified) if (!foundMatch) { // show diff with last approved