From 9fea2853c02e4628940b4d03ddc743a1795b48e1 Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Fri, 12 Feb 2021 11:34:29 -0800 Subject: [PATCH 01/16] code --- .../microsoft/hyperspace/index/Config.scala | 129 ++++++++++++++ .../hyperspace/index/ConfigBase.scala | 25 +++ .../hyperspace/index/IndexConfig.scala | 166 ------------------ .../index/configs/covering/IndexConfig.scala | 92 ++++++++++ .../noncovering/BloomFilterIndexConfig.scala | 24 +++ 5 files changed, 270 insertions(+), 166 deletions(-) create mode 100644 src/main/scala/com/microsoft/hyperspace/index/Config.scala create mode 100644 src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala delete mode 100644 src/main/scala/com/microsoft/hyperspace/index/IndexConfig.scala create mode 100644 src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala create mode 100644 src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala diff --git a/src/main/scala/com/microsoft/hyperspace/index/Config.scala b/src/main/scala/com/microsoft/hyperspace/index/Config.scala new file mode 100644 index 000000000..264dbb4e2 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/index/Config.scala @@ -0,0 +1,129 @@ +package com.microsoft.hyperspace.index + +import java.util.Locale + +/** + * IndexConfig specifies the configuration of an index. + * Associated Builder [[com.microsoft.hyperspace.index.configs.covering.IndexConfig.builder()]] + * + * @param indexName Index name. + * @param indexedColumns Columns from which an index is created. + * @param includedColumns Columns to be included in the index. + */ +case class IndexConfig( + indexName: String, + indexedColumns: Seq[String], + includedColumns: Seq[String] = Seq()) + extends CoveringIndexConfigBase { + if (indexName.isEmpty || indexedColumns.isEmpty) { + throw new IllegalArgumentException("Empty index name or indexed columns are not allowed.") + } + + val lowerCaseIndexedColumns: Seq[String] = toLowerCase(indexedColumns) + val lowerCaseIncludedColumns: Seq[String] = toLowerCase(includedColumns) + val lowerCaseIncludedColumnsSet: Set[String] = lowerCaseIncludedColumns.toSet + + if (lowerCaseIndexedColumns.toSet.size < lowerCaseIndexedColumns.size) { + throw new IllegalArgumentException("Duplicate indexed column names are not allowed.") + } + + if (lowerCaseIncludedColumnsSet.size < lowerCaseIncludedColumns.size) { + throw new IllegalArgumentException("Duplicate included column names are not allowed.") + } + + for (indexedColumn <- lowerCaseIndexedColumns) { + if (lowerCaseIncludedColumns.contains(indexedColumn)) { + throw new IllegalArgumentException( + "Duplicate column names in indexed/included columns are not allowed.") + } + } + + override def equals(that: Any): Boolean = { + that match { + case IndexConfig(thatIndexName, thatIndexedColumns, thatIncludedColumns) => + indexName.equalsIgnoreCase(thatIndexName) && + lowerCaseIndexedColumns.equals(toLowerCase(thatIndexedColumns)) && + lowerCaseIncludedColumnsSet.equals(toLowerCase(thatIncludedColumns).toSet) + case _ => false + } + } + + override def hashCode(): Int = { + lowerCaseIndexedColumns.hashCode + lowerCaseIncludedColumnsSet.hashCode + } + + override def toString: String = { + val indexedColumnNames = lowerCaseIndexedColumns.mkString(", ") + val includedColumnNames = lowerCaseIncludedColumns.mkString(", ") + s"[indexName: $indexName; indexedColumns: $indexedColumnNames; " + + s"includedColumns: $includedColumnNames]" + } + + private def toLowerCase(seq: Seq[String]): Seq[String] = seq.map(_.toLowerCase(Locale.ROOT)) +} + +/** + * TODO + * @param indexName + * @param indexedColumn + * @param expectedNumItems + * @param fpp + * @param numBits + */ +case class BloomFilterIndexConfig( + indexName: String, + indexedColumn: String, + expectedNumItems: Long, + fpp: Double = -1, + numBits: Long = -1) + extends NonCoveringIndexConfigBase { + if (indexName.isEmpty || indexedColumn.isEmpty || expectedNumItems < 1) { + throw new IllegalArgumentException( + "Empty index name or indexed column or expected items less than 1 are not allowed.") + } + + if (fpp != -1 && fpp < 0) { + throw new IllegalArgumentException("False positive probability cannot be negative.") + } + + if (numBits != -1 && numBits < 0) { + throw new IllegalArgumentException("Bits given for ") + } + + val lowerCaseIndexedColumn: String = indexedColumn.toLowerCase(Locale.ROOT) + + override def equals(that: Any): Boolean = { + that match { + case BloomFilterIndexConfig( + thatIndexName, + thatIndexedColumn, + thatExpectedItems, + thatFpp, + thatBits) => + indexName.equalsIgnoreCase(thatIndexName) && + lowerCaseIndexedColumn.equals(thatIndexedColumn.toLowerCase(Locale.ROOT)) && + expectedNumItems == thatExpectedItems && + ((fpp == -1 && thatFpp == -1) || fpp == thatFpp) && + ((numBits == -1 && thatBits == -1) || numBits == thatBits) + case _ => false + } + } + + override def hashCode(): Int = { + (indexName.hashCode * (indexedColumn.hashCode + expectedNumItems + .hashCode())) % scala.Int.MaxValue + } + + override def toString: String = { + s"[indexName: $indexName; indexedColumn: $indexedColumn; " + + s"ExpectedItems: $expectedNumItems; FPP: $fpp; NumBitsUsed: $numBits;]" + } +} + +object Config { + + // TODO - prints info table about all types of index supported by hyperspace + def printAllIndexConfigInfo(): String = { + s"" + } +} diff --git a/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala b/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala new file mode 100644 index 000000000..f0a597116 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala @@ -0,0 +1,25 @@ +package com.microsoft.hyperspace.index + +sealed trait IndexConfigBase { + val indexName: String + + def equals(obj: Any): Boolean + + def hashCode(): Int + + def toString: String +} + +trait CoveringIndexConfigBase extends IndexConfigBase { + /* + * Columns from which index are created. + */ + val indexedColumns: Seq[String] + + /* + * Columns to be included with the indexed columns. + */ + val includedColumns: Seq[String] +} + +trait NonCoveringIndexConfigBase extends IndexConfigBase {} diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexConfig.scala deleted file mode 100644 index 212606e66..000000000 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexConfig.scala +++ /dev/null @@ -1,166 +0,0 @@ -/* - * Copyright (2020) The Hyperspace Project Authors. - * - * Licensed 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 com.microsoft.hyperspace.index - -import java.util.Locale - -/** - * IndexConfig specifies the configuration of an index. - * - * @param indexName Index name. - * @param indexedColumns Columns from which an index is created. - * @param includedColumns Columns to be included in the index. - */ -case class IndexConfig( - indexName: String, - indexedColumns: Seq[String], - includedColumns: Seq[String] = Seq()) { - if (indexName.isEmpty || indexedColumns.isEmpty) { - throw new IllegalArgumentException("Empty index name or indexed columns are not allowed.") - } - - val lowerCaseIndexedColumns = toLowerCase(indexedColumns) - val lowerCaseIncludedColumns = toLowerCase(includedColumns) - val lowerCaseIncludedColumnsSet = lowerCaseIncludedColumns.toSet - - if (lowerCaseIndexedColumns.toSet.size < lowerCaseIndexedColumns.size) { - throw new IllegalArgumentException("Duplicate indexed column names are not allowed.") - } - - if (lowerCaseIncludedColumnsSet.size < lowerCaseIncludedColumns.size) { - throw new IllegalArgumentException("Duplicate included column names are not allowed.") - } - - for (indexedColumn <- lowerCaseIndexedColumns) { - if (lowerCaseIncludedColumns.contains(indexedColumn)) { - throw new IllegalArgumentException( - "Duplicate column names in indexed/included columns are not allowed.") - } - } - - override def equals(that: Any): Boolean = { - that match { - case IndexConfig(thatIndexName, thatIndexedColumns, thatIncludedColumns) => - indexName.equalsIgnoreCase(thatIndexName) && - lowerCaseIndexedColumns.equals(toLowerCase(thatIndexedColumns)) && - lowerCaseIncludedColumnsSet.equals(toLowerCase(thatIncludedColumns).toSet) - case _ => false - } - } - - override def hashCode(): Int = { - lowerCaseIndexedColumns.hashCode + lowerCaseIncludedColumnsSet.hashCode - } - - override def toString: String = { - val indexedColumnNames = lowerCaseIndexedColumns.mkString(", ") - val includedColumnNames = lowerCaseIncludedColumns.mkString(", ") - s"[indexName: $indexName; indexedColumns: $indexedColumnNames; " + - s"includedColumns: $includedColumnNames]" - } - - private def toLowerCase(seq: Seq[String]): Seq[String] = seq.map(_.toLowerCase(Locale.ROOT)) -} - -/** - * Defines [[IndexConfig.Builder]] and relevant helper methods for enabling builder pattern for - * [[IndexConfig]]. - */ -object IndexConfig { - - /** - * Builder for [[IndexConfig]]. - */ - class Builder { - - private[this] var indexedColumns: Seq[String] = Seq() - private[this] var includedColumns: Seq[String] = Seq() - private[this] var indexName: String = "" - - /** - * Updates index name for [[IndexConfig]]. - * - * @param indexName index name for the [[IndexConfig]]. - * @return an [[IndexConfig.Builder]] object with updated index name. - */ - def indexName(indexName: String): Builder = { - if (!this.indexName.isEmpty) { - throw new UnsupportedOperationException("Index name is already set.") - } - - if (indexName.isEmpty) { - throw new IllegalArgumentException("Empty index name is not allowed.") - } - - this.indexName = indexName - this - } - - /** - * Updates column names for [[IndexConfig]]. - * - * Note: API signature supports passing one or more argument. - * - * @param indexedColumn indexed column for the [[IndexConfig]]. - * @param indexedColumns indexed columns for the [[IndexConfig]]. - * @return an [[IndexConfig.Builder]] object with updated indexed columns. - */ - def indexBy(indexedColumn: String, indexedColumns: String*): Builder = { - if (this.indexedColumns.nonEmpty) { - throw new UnsupportedOperationException("Indexed columns are already set.") - } - - this.indexedColumns = indexedColumn +: indexedColumns - this - } - - /** - * Updates included columns for [[IndexConfig]]. - * - * Note: API signature supports passing one or more argument. - * - * @param includedColumn included column for [[IndexConfig]]. - * @param includedColumns included columns for [[IndexConfig]]. - * @return an [[IndexConfig.Builder]] object with updated included columns. - */ - def include(includedColumn: String, includedColumns: String*): Builder = { - if (this.includedColumns.nonEmpty) { - throw new UnsupportedOperationException("Included columns are already set.") - } - - this.includedColumns = includedColumn +: includedColumns - this - } - - /** - * Creates IndexConfig from supplied index name, indexed columns and included columns - * to [[IndexConfig.Builder]]. - * - * @return an [[IndexConfig]] object. - */ - def create(): IndexConfig = { - IndexConfig(indexName, indexedColumns, includedColumns) - } - } - - /** - * Creates new [[IndexConfig.Builder]] for constructing an [[IndexConfig]]. - * - * @return an [[IndexConfig.Builder]] object. - */ - def builder(): Builder = new Builder -} diff --git a/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala new file mode 100644 index 000000000..f9871338d --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala @@ -0,0 +1,92 @@ +package com.microsoft.hyperspace.index.configs.covering + +import com.microsoft.hyperspace.index.IndexConfig + +/** + * Defines [[IndexConfig.Builder]] and relevant helper methods for enabling builder pattern for + * [[IndexConfig]]. + */ +object IndexConfig { + + /** + * Builder for [[IndexConfig]]. + */ + private[index] class Builder { + + private[this] var indexedColumns: Seq[String] = Seq() + private[this] var includedColumns: Seq[String] = Seq() + private[this] var indexName: String = "" + + /** + * Updates index name for [[IndexConfig]]. + * + * @param indexName index name for the [[IndexConfig]]. + * @return an [[IndexConfig.Builder]] object with updated index name. + */ + def indexName(indexName: String): Builder = { + if (this.indexName.nonEmpty) { + throw new UnsupportedOperationException("Index name is already set.") + } + + if (indexName.isEmpty) { + throw new IllegalArgumentException("Empty index name is not allowed.") + } + + this.indexName = indexName + this + } + + /** + * Updates column names for [[IndexConfig]]. + * + * Note: API signature supports passing one or more argument. + * + * @param indexedColumn indexed column for the [[IndexConfig]]. + * @param indexedColumns indexed columns for the [[IndexConfig]]. + * @return an [[IndexConfig.Builder]] object with updated indexed columns. + */ + def indexBy(indexedColumn: String, indexedColumns: String*): Builder = { + if (this.indexedColumns.nonEmpty) { + throw new UnsupportedOperationException("Indexed columns are already set.") + } + + this.indexedColumns = indexedColumn +: indexedColumns + this + } + + /** + * Updates included columns for [[IndexConfig]]. + * + * Note: API signature supports passing one or more argument. + * + * @param includedColumn included column for [[IndexConfig]]. + * @param includedColumns included columns for [[IndexConfig]]. + * @return an [[IndexConfig.Builder]] object with updated included columns. + */ + def include(includedColumn: String, includedColumns: String*): Builder = { + if (this.includedColumns.nonEmpty) { + throw new UnsupportedOperationException("Included columns are already set.") + } + + this.includedColumns = includedColumn +: includedColumns + this + } + + /** + * Creates IndexConfig from supplied index name, indexed columns and included columns + * to [[IndexConfig.Builder]]. + * + * @return an [[IndexConfig]] object. + */ + def build(): IndexConfig = { + new IndexConfig(indexName, indexedColumns, includedColumns) + } + } + + /** + * Creates new [[IndexConfig.Builder]] for constructing an [[IndexConfig]]. + * + * @return an [[IndexConfig.Builder]] object. + */ + def builder(): Builder = new Builder +} diff --git a/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala new file mode 100644 index 000000000..7e39fe489 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala @@ -0,0 +1,24 @@ +package com.microsoft.hyperspace.index.configs.noncovering + +/** + * TODO Defines [[BloomFilterIndexConfig.Builder]] and relevant helper methods for enabling builder pattern for + * [[BloomFilterIndexConfig]]. + */ +object BloomFilterIndexConfig { + + /** + * Builder for [[BloomFilterIndexConfig]]. + */ + private[index] class Builder { + + private[this] var indexedColumn: String = "" + private[this] var indexName: String = "" + } + + /** + * Creates new [[BloomFilterIndexConfig.Builder]] for constructing an [[BloomFilterIndexConfig]]. + * + * @return an [[BloomFilterIndexConfig.Builder]] object. + */ + def builder(): Builder = new Builder +} From 5d6b6cb434dd66127d2ae97294c453123b47efc7 Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Fri, 12 Feb 2021 12:32:11 -0800 Subject: [PATCH 02/16] test fixed for build --- .../microsoft/hyperspace/index/Config.scala | 16 +++++----- .../hyperspace/index/ConfigBase.scala | 10 ++++-- .../index/configs/covering/IndexConfig.scala | 2 +- .../noncovering/BloomFilterIndexConfig.scala | 4 +-- .../hyperspace/index/IndexConfigTest.scala | 32 +++++++++++-------- 5 files changed, 37 insertions(+), 27 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/Config.scala b/src/main/scala/com/microsoft/hyperspace/index/Config.scala index 264dbb4e2..8bf3fe273 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/Config.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/Config.scala @@ -2,6 +2,14 @@ package com.microsoft.hyperspace.index import java.util.Locale +object Config { + + // TODO - prints info table about all types of index supported by hyperspace + def printAllIndexConfigInfo(): String = { + s"" + } +} + /** * IndexConfig specifies the configuration of an index. * Associated Builder [[com.microsoft.hyperspace.index.configs.covering.IndexConfig.builder()]] @@ -119,11 +127,3 @@ case class BloomFilterIndexConfig( s"ExpectedItems: $expectedNumItems; FPP: $fpp; NumBitsUsed: $numBits;]" } } - -object Config { - - // TODO - prints info table about all types of index supported by hyperspace - def printAllIndexConfigInfo(): String = { - s"" - } -} diff --git a/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala b/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala index f0a597116..fe92c9ed1 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala @@ -1,6 +1,10 @@ package com.microsoft.hyperspace.index -sealed trait IndexConfigBase { +/** + * All index supported in Hyperspace whose user facing config needs to be defined needs + * to extend [[ConfigBase]] trait. + */ +sealed trait ConfigBase { val indexName: String def equals(obj: Any): Boolean @@ -10,7 +14,7 @@ sealed trait IndexConfigBase { def toString: String } -trait CoveringIndexConfigBase extends IndexConfigBase { +trait CoveringIndexConfigBase extends ConfigBase { /* * Columns from which index are created. */ @@ -22,4 +26,4 @@ trait CoveringIndexConfigBase extends IndexConfigBase { val includedColumns: Seq[String] } -trait NonCoveringIndexConfigBase extends IndexConfigBase {} +trait NonCoveringIndexConfigBase extends ConfigBase {} diff --git a/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala index f9871338d..0882e2d28 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala @@ -11,7 +11,7 @@ object IndexConfig { /** * Builder for [[IndexConfig]]. */ - private[index] class Builder { + class Builder { private[this] var indexedColumns: Seq[String] = Seq() private[this] var includedColumns: Seq[String] = Seq() diff --git a/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala index 7e39fe489..492426c4f 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala @@ -1,8 +1,8 @@ package com.microsoft.hyperspace.index.configs.noncovering /** - * TODO Defines [[BloomFilterIndexConfig.Builder]] and relevant helper methods for enabling builder pattern for - * [[BloomFilterIndexConfig]]. + * TODO Defines [[BloomFilterIndexConfig.Builder]] and relevant helper methods for enabling + * builder pattern for [[BloomFilterIndexConfig]]. */ object BloomFilterIndexConfig { diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala index cd848b86e..66169756f 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala @@ -18,29 +18,35 @@ package com.microsoft.hyperspace.index import org.apache.spark.SparkFunSuite +import com.microsoft.hyperspace.index.configs.covering + class IndexConfigTest extends SparkFunSuite { + val CoveringIndexBuilder: covering.IndexConfig.type = covering.IndexConfig + test("Empty index name is not allowed.") { + intercept[IllegalArgumentException](IndexConfig("", Seq("c1"), Seq("c2"))) - intercept[IllegalArgumentException](IndexConfig.builder.indexBy("c1").include("c2").create) - intercept[IllegalArgumentException](IndexConfig.builder.indexName("")) + intercept[IllegalArgumentException]( + CoveringIndexBuilder.builder().indexBy("c1").include("c2").build()) + intercept[IllegalArgumentException](CoveringIndexBuilder.builder().indexName("")) } test("Empty indexed columns are not allowed.") { intercept[IllegalArgumentException](IndexConfig("name", Seq(), Seq("c1"))) intercept[IllegalArgumentException]( - IndexConfig.builder.indexName("name").include("c1").create) + CoveringIndexBuilder.builder().indexName("name").include("c1").build()) } test("Same indexed column names (case-insensitive) are not allowed.") { intercept[IllegalArgumentException](IndexConfig("name", Seq("c1", "C1"), Seq("c2"))) intercept[IllegalArgumentException]( - IndexConfig.builder.indexName("name").indexBy("c1", "C1").include("c2").create) + CoveringIndexBuilder.builder().indexName("name").indexBy("c1", "C1").include("c2").build()) } test("Same column names (case-insensitive) in indexed/included columns are not allowed.") { intercept[IllegalArgumentException](IndexConfig("name", Seq("c1"), Seq("C1", "c2"))) intercept[IllegalArgumentException]( - IndexConfig.builder.indexName("name").indexBy("c1").include("C1", "c2").create) + CoveringIndexBuilder.builder().indexName("name").indexBy("c1").include("C1", "c2").build()) } test("Test equals() function.") { @@ -98,11 +104,11 @@ class IndexConfigTest extends SparkFunSuite { val indexedColumns = Seq("C1", "c2", "C3") val includedColumns = Seq("C4", "c5", "C6") - val indexConfig = IndexConfig.builder + val indexConfig = CoveringIndexBuilder.builder() .indexName(indexName) .indexBy(indexedColumns.head, indexedColumns.tail: _*) .include(includedColumns.head, includedColumns.tail: _*) - .create + .build() assert(indexConfig.indexName.equals(indexName)) assert(indexConfig.indexedColumns.equals(indexedColumns)) @@ -111,25 +117,25 @@ class IndexConfigTest extends SparkFunSuite { test("Test exception on multiple indexBy, include and index name on IndexConfig builder.") { intercept[UnsupportedOperationException]( - IndexConfig.builder + CoveringIndexBuilder.builder() .indexName("name1") .indexName("name2") .indexBy("c1", "c2") .include("c3", "c4") - .create) + .build()) intercept[UnsupportedOperationException]( - IndexConfig.builder + CoveringIndexBuilder.builder() .indexName("name") .indexBy("c1") .indexBy("c2") .include("c3", "c4") - .create) + .build()) intercept[UnsupportedOperationException]( - IndexConfig.builder + CoveringIndexBuilder.builder() .indexName("name") .indexBy("c1") .include("c2") .include("c3") - .create) + .build()) } } From ed55d418a5aea3932235e2e07b7f555f7a702d80 Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Fri, 12 Feb 2021 13:20:17 -0800 Subject: [PATCH 03/16] header --- .../microsoft/hyperspace/index/Config.scala | 40 +++++++++++-------- .../hyperspace/index/ConfigBase.scala | 16 ++++++++ .../index/configs/covering/IndexConfig.scala | 16 ++++++++ .../noncovering/BloomFilterIndexConfig.scala | 16 ++++++++ 4 files changed, 72 insertions(+), 16 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/Config.scala b/src/main/scala/com/microsoft/hyperspace/index/Config.scala index 8bf3fe273..3ff5ec807 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/Config.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/Config.scala @@ -1,15 +1,23 @@ +/* + * Copyright (2020) The Hyperspace Project Authors. + * + * Licensed 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 com.microsoft.hyperspace.index import java.util.Locale -object Config { - - // TODO - prints info table about all types of index supported by hyperspace - def printAllIndexConfigInfo(): String = { - s"" - } -} - /** * IndexConfig specifies the configuration of an index. * Associated Builder [[com.microsoft.hyperspace.index.configs.covering.IndexConfig.builder()]] @@ -70,14 +78,6 @@ case class IndexConfig( private def toLowerCase(seq: Seq[String]): Seq[String] = seq.map(_.toLowerCase(Locale.ROOT)) } -/** - * TODO - * @param indexName - * @param indexedColumn - * @param expectedNumItems - * @param fpp - * @param numBits - */ case class BloomFilterIndexConfig( indexName: String, indexedColumn: String, @@ -127,3 +127,11 @@ case class BloomFilterIndexConfig( s"ExpectedItems: $expectedNumItems; FPP: $fpp; NumBitsUsed: $numBits;]" } } + +object Config { + + // TODO - prints info table about all types of index supported by hyperspace + def printAllIndexConfigInfo(): String = { + s"" + } +} diff --git a/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala b/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala index fe92c9ed1..4e1e8018a 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala @@ -1,3 +1,19 @@ +/* + * Copyright (2020) The Hyperspace Project Authors. + * + * Licensed 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 com.microsoft.hyperspace.index /** diff --git a/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala index 0882e2d28..838966562 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala @@ -1,3 +1,19 @@ +/* + * Copyright (2020) The Hyperspace Project Authors. + * + * Licensed 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 com.microsoft.hyperspace.index.configs.covering import com.microsoft.hyperspace.index.IndexConfig diff --git a/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala index 492426c4f..ceed6f68c 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala @@ -1,3 +1,19 @@ +/* + * Copyright (2020) The Hyperspace Project Authors. + * + * Licensed 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 com.microsoft.hyperspace.index.configs.noncovering /** From 4ee368c36bb89b9183a2d0ce5504fda08009650c Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Wed, 17 Feb 2021 17:25:48 -0800 Subject: [PATCH 04/16] code --- .../noncovering/BloomFilterIndexConfig.scala | 75 +++++++++++++++++++ .../hyperspace/index/IndexConfigTest.scala | 34 ++++++--- 2 files changed, 99 insertions(+), 10 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala index ceed6f68c..8ab24a9ed 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala @@ -16,6 +16,9 @@ package com.microsoft.hyperspace.index.configs.noncovering +import com.microsoft.hyperspace.index.BloomFilterIndexConfig +import com.microsoft.hyperspace.index.configs.covering.IndexConfig + /** * TODO Defines [[BloomFilterIndexConfig.Builder]] and relevant helper methods for enabling * builder pattern for [[BloomFilterIndexConfig]]. @@ -29,6 +32,78 @@ object BloomFilterIndexConfig { private[this] var indexedColumn: String = "" private[this] var indexName: String = "" + private[this] var fpp: Double = -1 + private[this] var expectedItems: Long = -1 + private[this] var numBits: Long = -1 + + /** + * Updates index name for [[IndexConfig]]. + * + * @param indexName index name for the [[BloomFilterIndexConfig]]. + * @return an [[BloomFilterIndexConfig.Builder]] object with updated index name. + */ + def init(indexName: String, indexedColumn: String): Builder = { + if (this.indexName.nonEmpty || this.indexedColumn.nonEmpty) { + // TODO: Prevent creating index config if index already exists. + throw new UnsupportedOperationException( + "Bloom Filter index metadata already set can't override, " + + "maybe try creating a new config.") + } + + if (indexName.isEmpty || indexedColumn.isEmpty) { + throw new IllegalArgumentException("Empty metadata names is not allowed.") + } + + this.indexName = indexName + this.indexedColumn = indexedColumn + this + } + + /** + * + * @param items + * @return + */ + def expectedNumItems(items: Long): Builder = { + if (items < 1) { + throw new IllegalArgumentException("Can't support the items value provided.") + } + + this.expectedItems = items + this + } + + /** + * + * @param fpp + * @return + */ + def fppToSupport(fpp: Double): Builder = { + if (fpp <= 0) { + throw new IllegalArgumentException("Can't support the fpp value.") + } + + this.fpp = fpp + this + } + + /** + * + * @param bits + * @return + */ + def numBitsToDefineBloomFilter(bits: Long): Builder = { + if (bits < 1) { + throw new IllegalArgumentException("Can't allow bits for storage be less than 1") + } + + this.numBits = bits + this + } + + def build(): BloomFilterIndexConfig = { + new BloomFilterIndexConfig(indexName, indexedColumn, expectedItems, fpp, numBits) + } } /** diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala index 66169756f..5cf769f81 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala @@ -21,32 +21,42 @@ import org.apache.spark.SparkFunSuite import com.microsoft.hyperspace.index.configs.covering class IndexConfigTest extends SparkFunSuite { - val CoveringIndexBuilder: covering.IndexConfig.type = covering.IndexConfig + val CoveringIndexConfigBuilder: covering.IndexConfig.type = covering.IndexConfig test("Empty index name is not allowed.") { intercept[IllegalArgumentException](IndexConfig("", Seq("c1"), Seq("c2"))) intercept[IllegalArgumentException]( - CoveringIndexBuilder.builder().indexBy("c1").include("c2").build()) - intercept[IllegalArgumentException](CoveringIndexBuilder.builder().indexName("")) + CoveringIndexConfigBuilder.builder().indexBy("c1").include("c2").build()) + intercept[IllegalArgumentException](CoveringIndexConfigBuilder.builder().indexName("")) } test("Empty indexed columns are not allowed.") { intercept[IllegalArgumentException](IndexConfig("name", Seq(), Seq("c1"))) intercept[IllegalArgumentException]( - CoveringIndexBuilder.builder().indexName("name").include("c1").build()) + CoveringIndexConfigBuilder.builder().indexName("name").include("c1").build()) } test("Same indexed column names (case-insensitive) are not allowed.") { intercept[IllegalArgumentException](IndexConfig("name", Seq("c1", "C1"), Seq("c2"))) intercept[IllegalArgumentException]( - CoveringIndexBuilder.builder().indexName("name").indexBy("c1", "C1").include("c2").build()) + CoveringIndexConfigBuilder + .builder() + .indexName("name") + .indexBy("c1", "C1") + .include("c2") + .build()) } test("Same column names (case-insensitive) in indexed/included columns are not allowed.") { intercept[IllegalArgumentException](IndexConfig("name", Seq("c1"), Seq("C1", "c2"))) intercept[IllegalArgumentException]( - CoveringIndexBuilder.builder().indexName("name").indexBy("c1").include("C1", "c2").build()) + CoveringIndexConfigBuilder + .builder() + .indexName("name") + .indexBy("c1") + .include("C1", "c2") + .build()) } test("Test equals() function.") { @@ -104,7 +114,8 @@ class IndexConfigTest extends SparkFunSuite { val indexedColumns = Seq("C1", "c2", "C3") val includedColumns = Seq("C4", "c5", "C6") - val indexConfig = CoveringIndexBuilder.builder() + val indexConfig = CoveringIndexConfigBuilder + .builder() .indexName(indexName) .indexBy(indexedColumns.head, indexedColumns.tail: _*) .include(includedColumns.head, includedColumns.tail: _*) @@ -117,21 +128,24 @@ class IndexConfigTest extends SparkFunSuite { test("Test exception on multiple indexBy, include and index name on IndexConfig builder.") { intercept[UnsupportedOperationException]( - CoveringIndexBuilder.builder() + CoveringIndexConfigBuilder + .builder() .indexName("name1") .indexName("name2") .indexBy("c1", "c2") .include("c3", "c4") .build()) intercept[UnsupportedOperationException]( - CoveringIndexBuilder.builder() + CoveringIndexConfigBuilder + .builder() .indexName("name") .indexBy("c1") .indexBy("c2") .include("c3", "c4") .build()) intercept[UnsupportedOperationException]( - CoveringIndexBuilder.builder() + CoveringIndexConfigBuilder + .builder() .indexName("name") .indexBy("c1") .include("c2") From e0ee520b256b6c01c48d5293409e9009de9d9d92 Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Mon, 22 Feb 2021 15:47:12 -0800 Subject: [PATCH 05/16] 1 --- .../hyperspace/index/IndexLogEntry.scala | 46 +++++++++++++++++-- .../hyperspace/index/IndexStatistics.scala | 2 +- .../plans/logical/IndexHadoopFsRelation.scala | 2 +- 3 files changed, 45 insertions(+), 5 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index 58817e1e6..7c410244f 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -343,17 +343,57 @@ object FileInfo { } } +trait HyperSpaceIndex { + def getKind(): String + + def getKindAbbr(): String +} + +trait NonClusteredIndex extends HyperSpaceIndex { + object Kinds extends Enumeration { + type Kinds = String + val covering = "CoveringIndex" + val nonCovering = "NonCoveringIndex" + } +} + // IndexLogEntry-specific CoveringIndex that represents derived dataset. -case class CoveringIndex(properties: CoveringIndex.Properties) { - val kind = "CoveringIndex" - val kindAbbr = "CI" +case class CoveringIndex(properties: CoveringIndex.Properties) extends NonClusteredIndex { + private val kind: String = Kinds.covering + private val kindAbbr: String = "CI" + + override def getKind(): String = { + this.kind + } + + override def getKindAbbr(): String = { + this.kindAbbr + } } + +case class BloomFilterIndex(properties: CoveringIndex.BFProperties) extends NonClusteredIndex { + private val kind: String = Kinds.nonCovering + private val kindAbbr: String = "BF" + + override def getKind(): String = { + this.kind + } + + override def getKindAbbr(): String = { + this.kindAbbr + } +} + object CoveringIndex { case class Properties(columns: Properties.Columns, schemaString: String, numBuckets: Int, properties: Map[String, String]) + case class BFProperties(columns: Properties.Columns, + schemaString: String, + properties: Map[String, String]) + object Properties { case class Columns(indexed: Seq[String], included: Seq[String]) } diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexStatistics.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexStatistics.scala index 2d836ba41..654593574 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexStatistics.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexStatistics.scala @@ -88,7 +88,7 @@ private[hyperspace] object IndexStatistics { entry.derivedDataset.properties.schemaString, indexDirPath(entry), entry.state, - entry.derivedDataset.kind, + entry.derivedDataset.getKind(), entry.hasLineageColumn, entry.content.fileInfos.size, entry.content.fileInfos.foldLeft(0L)(_ + _.size), diff --git a/src/main/scala/com/microsoft/hyperspace/index/plans/logical/IndexHadoopFsRelation.scala b/src/main/scala/com/microsoft/hyperspace/index/plans/logical/IndexHadoopFsRelation.scala index d46b25fb6..1a26dc608 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/plans/logical/IndexHadoopFsRelation.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/plans/logical/IndexHadoopFsRelation.scala @@ -42,7 +42,7 @@ class IndexHadoopFsRelation( options)(spark) { val indexPlanStr: String = { - s"Hyperspace(Type: ${index.derivedDataset.kindAbbr}, " + + s"Hyperspace(Type: ${index.derivedDataset.getKindAbbr()}, " + s"Name: ${index.name}, LogVersion: ${index.id})" } override def toString(): String = indexPlanStr From ef3f8874ed1cc9ec91b5b9db8b40b4ff6a408946 Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Mon, 22 Feb 2021 19:23:57 -0800 Subject: [PATCH 06/16] 2 --- .../com/microsoft/hyperspace/Hyperspace.scala | 2 +- .../hyperspace/actions/CreateAction.scala | 16 ++- .../hyperspace/actions/CreateActionBase.scala | 88 ++++++++++---- .../index/CachingIndexCollectionManager.scala | 2 +- .../hyperspace/index/ConfigBase.scala | 45 ------- ...fig.scala => HyperSpaceIndexConfigs.scala} | 38 +++++- .../index/IndexCollectionManager.scala | 2 +- .../hyperspace/index/IndexLogEntry.scala | 112 ++++++++++-------- .../hyperspace/index/IndexManager.scala | 2 +- .../hyperspace/index/IndexStatistics.scala | 2 +- ...exConfig.scala => BloomFilterConfig.scala} | 21 ++-- ...IndexConfig.scala => CoveringConfig.scala} | 42 +++---- .../plans/logical/IndexHadoopFsRelation.scala | 2 +- .../actions/RefreshActionTest.scala | 6 +- .../hyperspace/index/IndexCacheTest.scala | 6 +- .../index/IndexCollectionManagerTest.scala | 12 +- .../hyperspace/index/IndexConfigTest.scala | 4 +- .../hyperspace/index/IndexLogEntryTest.scala | 6 +- .../index/IndexLogManagerImplTest.scala | 6 +- .../hyperspace/index/IndexManagerTest.scala | 6 +- .../hyperspace/index/IndexTest.scala | 6 +- .../index/rules/HyperspaceRuleSuite.scala | 6 +- .../hyperspace/util/JsonUtilsTest.scala | 6 +- 23 files changed, 241 insertions(+), 197 deletions(-) delete mode 100644 src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala rename src/main/scala/com/microsoft/hyperspace/index/{Config.scala => HyperSpaceIndexConfigs.scala} (85%) rename src/main/scala/com/microsoft/hyperspace/index/configs/{noncovering/BloomFilterIndexConfig.scala => BloomFilterConfig.scala} (78%) rename src/main/scala/com/microsoft/hyperspace/index/configs/{covering/IndexConfig.scala => CoveringConfig.scala} (64%) diff --git a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala index 4eaa07b90..af5e90dc9 100644 --- a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala +++ b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala @@ -39,7 +39,7 @@ class Hyperspace(spark: SparkSession) { * @param df the DataFrame object to build index on. * @param indexConfig the configuration of index to be created. */ - def createIndex(df: DataFrame, indexConfig: IndexConfig): Unit = { + def createIndex(df: DataFrame, indexConfig: HyperSpaceIndexConfig): Unit = { indexManager.create(df, indexConfig) } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index b81deed86..17b05991d 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -16,10 +16,9 @@ package com.microsoft.hyperspace.actions -import scala.util.Try - import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.types.StructType +import scala.util.Try import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, CREATING, DOESNOTEXIST} @@ -30,7 +29,7 @@ import com.microsoft.hyperspace.util.ResolverUtils class CreateAction( spark: SparkSession, df: DataFrame, - indexConfig: IndexConfig, + indexConfig: HyperSpaceIndexConfig, final override protected val logManager: IndexLogManager, dataManager: IndexDataManager) extends CreateActionBase(dataManager) @@ -50,7 +49,7 @@ class CreateAction( } // schema validity checks - if (!isValidIndexSchema(indexConfig, df.schema)) { + if (!isValidIndexSchema(indexConfig.asInstanceOf[IndexConfig], df.schema)) { throw HyperspaceException("Index config is not applicable to dataframe schema.") } @@ -73,11 +72,16 @@ class CreateAction( // TODO: The following should be protected, but RefreshAction is calling CreateAction.op(). // This needs to be refactored to mark this as protected. - final override def op(): Unit = write(spark, df, indexConfig) + final override def op(): Unit = write(spark, df, indexConfig.asInstanceOf[IndexConfig]) final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { // LogEntry instantiation may fail if index config is invalid. Hence the 'Try'. val index = Try(logEntry.asInstanceOf[IndexLogEntry]).toOption - CreateActionEvent(appInfo, indexConfig, index, df.queryExecution.logical.toString, message) + CreateActionEvent( + appInfo, + indexConfig.asInstanceOf[IndexConfig], + index, + df.queryExecution.logical.toString, + message) } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 08d875e37..ea0a47315 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -51,7 +51,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) protected def getIndexLogEntry( spark: SparkSession, df: DataFrame, - indexConfig: IndexConfig, + indexConfig: HyperSpaceIndexConfig, path: Path): IndexLogEntry = { val absolutePath = PathUtils.makeAbsolute(path, spark.sessionState.newHadoopConf()) val numBuckets = numBucketsForIndex(spark) @@ -72,18 +72,29 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) LogicalPlanFingerprint( LogicalPlanFingerprint.Properties(Seq(Signature(signatureProvider.name, s))))) - val coveringIndexProperties = + val indexProperties = (hasLineageProperty(spark) ++ hasParquetAsSourceFormatProperty(relation)).toMap IndexLogEntry( indexConfig.indexName, - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties - .Columns(resolvedIndexedColumns, resolvedIncludedColumns), - IndexLogEntry.schemaString(indexDataFrame.schema), - numBuckets, - coveringIndexProperties)), + indexConfig match { + case _: IndexConfig => + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties + .Columns(resolvedIndexedColumns, resolvedIncludedColumns), + IndexLogEntry.schemaString(indexDataFrame.schema), + numBuckets, + indexProperties)) + case _: HyperSpaceIndexConfig => + HyperSpaceIndex.BloomFilterIndex( + HyperSpaceIndex.Properties.BloomFilter( + HyperSpaceIndex.Properties.CommonProperties + .Columns(resolvedIncludedColumns, resolvedIncludedColumns), + IndexLogEntry.schemaString(indexDataFrame.schema), + indexProperties)) + case _ => throw HyperspaceException("Invalid Index Config.") + }, Content.fromDirectory(absolutePath, fileIdTracker), Source(SparkPlan(sourcePlanProperties)), Map()) @@ -140,7 +151,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) relations.head } - private def resolveConfig( + private def resolveCoveringIndexConfig( df: DataFrame, indexConfig: IndexConfig): (Seq[String], Seq[String]) = { val spark = df.sparkSession @@ -162,14 +173,28 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) } } - private def prepareIndexDataFrame( - spark: SparkSession, + private def resolveBloomFilterIndexConfig( df: DataFrame, - indexConfig: IndexConfig): (DataFrame, Seq[String], Seq[String]) = { - val (resolvedIndexedColumns, resolvedIncludedColumns) = resolveConfig(df, indexConfig) - val columnsFromIndexConfig = resolvedIndexedColumns ++ resolvedIncludedColumns + indexConfig: BloomFilterIndexConfig): String = { + val spark = df.sparkSession + val dfColumnNames = df.schema.fieldNames + val indexedColumn = indexConfig.indexedColumn + val resolvedIndexedColumns = ResolverUtils.resolve(spark, indexedColumn, dfColumnNames) + + resolvedIndexedColumns match { + case Some(indexed) => indexed + case _ => + throw HyperspaceException( + s"Columns '${indexedColumn}' could not be resolved " + + s"from available source columns '${dfColumnNames.mkString(",")}'") + } + } - val indexDF = if (hasLineage(spark)) { + private def resolveDataFrameForLineage( + spark: SparkSession, + df: DataFrame, + providedIndexColumns: Seq[String]): DataFrame = { + if (hasLineage(spark)) { val relation = getRelation(spark, df) // Lineage is captured using two sets of columns: @@ -177,9 +202,9 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) // 2. If source data is partitioned, all partitioning key(s) are added to index schema // as columns if they are not already part of the schema. val partitionColumns = relation.partitionSchema.map(_.name) - val missingPartitionColumns = partitionColumns.filter( - ResolverUtils.resolve(spark, _, columnsFromIndexConfig).isEmpty) - val allIndexColumns = columnsFromIndexConfig ++ missingPartitionColumns + val missingPartitionColumns = + partitionColumns.filter(ResolverUtils.resolve(spark, _, providedIndexColumns).isEmpty) + val allIndexColumns = providedIndexColumns ++ missingPartitionColumns // File id value in DATA_FILE_ID_COLUMN column (lineage column) is stored as a // Long data type value. Each source data file has a unique file id, assigned by @@ -202,10 +227,29 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) .select( allIndexColumns.head, allIndexColumns.tail :+ IndexConstants.DATA_FILE_NAME_ID: _*) - } else { - df.select(columnsFromIndexConfig.head, columnsFromIndexConfig.tail: _*) } - (indexDF, resolvedIndexedColumns, resolvedIncludedColumns) + df.select(providedIndexColumns.head, providedIndexColumns.tail: _*) + } + + private def prepareIndexDataFrame( + spark: SparkSession, + df: DataFrame, + indexConfig: HyperSpaceIndexConfig): (DataFrame, Seq[String], Seq[String]) = { + + indexConfig match { + case coveringIndexConfig: IndexConfig => + val (resolvedIndexedColumns, resolvedIncludedColumns) = + resolveCoveringIndexConfig(df, coveringIndexConfig) + val columnsFromIndexConfig = resolvedIndexedColumns ++ resolvedIncludedColumns + val indexDF = resolveDataFrameForLineage(spark, df, columnsFromIndexConfig) + + (indexDF, resolvedIndexedColumns, resolvedIncludedColumns) + case bloomIndexConfig: BloomFilterIndexConfig => + val resolvedIndexColumn = Seq(resolveBloomFilterIndexConfig(df, bloomIndexConfig)) + val indexDF = resolveDataFrameForLineage(spark, df, resolvedIndexColumn) + + (indexDF, resolvedIndexColumn, Seq()) + } } } diff --git a/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala index a1a3c9e8d..49891e62a 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala @@ -73,7 +73,7 @@ class CachingIndexCollectionManager( indexCache.clear() } - override def create(df: DataFrame, indexConfig: IndexConfig): Unit = { + override def create(df: DataFrame, indexConfig: HyperSpaceIndexConfig): Unit = { clearCache() super.create(df, indexConfig) } diff --git a/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala b/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala deleted file mode 100644 index 4e1e8018a..000000000 --- a/src/main/scala/com/microsoft/hyperspace/index/ConfigBase.scala +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Copyright (2020) The Hyperspace Project Authors. - * - * Licensed 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 com.microsoft.hyperspace.index - -/** - * All index supported in Hyperspace whose user facing config needs to be defined needs - * to extend [[ConfigBase]] trait. - */ -sealed trait ConfigBase { - val indexName: String - - def equals(obj: Any): Boolean - - def hashCode(): Int - - def toString: String -} - -trait CoveringIndexConfigBase extends ConfigBase { - /* - * Columns from which index are created. - */ - val indexedColumns: Seq[String] - - /* - * Columns to be included with the indexed columns. - */ - val includedColumns: Seq[String] -} - -trait NonCoveringIndexConfigBase extends ConfigBase {} diff --git a/src/main/scala/com/microsoft/hyperspace/index/Config.scala b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfigs.scala similarity index 85% rename from src/main/scala/com/microsoft/hyperspace/index/Config.scala rename to src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfigs.scala index 3ff5ec807..f8de8b19f 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/Config.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfigs.scala @@ -18,9 +18,39 @@ package com.microsoft.hyperspace.index import java.util.Locale +import com.microsoft.hyperspace.index.configs.CoveringConfig + +/** + * All index supported in Hyperspace whose user facing config needs to be defined needs + * to extend [[HyperSpaceIndexConfig]] trait. + */ +sealed trait HyperSpaceIndexConfig { + val indexName: String + + def equals(obj: Any): Boolean + + def hashCode(): Int + + def toString: String +} + +trait CoveringIndexConfig extends HyperSpaceIndexConfig { + /* + * Columns from which index are created. + */ + val indexedColumns: Seq[String] + + /* + * Columns to be included with the indexed columns. + */ + val includedColumns: Seq[String] +} + +trait NonCoveringIndexConfig extends HyperSpaceIndexConfig {} + /** * IndexConfig specifies the configuration of an index. - * Associated Builder [[com.microsoft.hyperspace.index.configs.covering.IndexConfig.builder()]] + * Associated Builder [[CoveringConfig.builder()]] * * @param indexName Index name. * @param indexedColumns Columns from which an index is created. @@ -30,7 +60,7 @@ case class IndexConfig( indexName: String, indexedColumns: Seq[String], includedColumns: Seq[String] = Seq()) - extends CoveringIndexConfigBase { + extends CoveringIndexConfig { if (indexName.isEmpty || indexedColumns.isEmpty) { throw new IllegalArgumentException("Empty index name or indexed columns are not allowed.") } @@ -84,7 +114,7 @@ case class BloomFilterIndexConfig( expectedNumItems: Long, fpp: Double = -1, numBits: Long = -1) - extends NonCoveringIndexConfigBase { + extends NonCoveringIndexConfig { if (indexName.isEmpty || indexedColumn.isEmpty || expectedNumItems < 1) { throw new IllegalArgumentException( "Empty index name or indexed column or expected items less than 1 are not allowed.") @@ -128,7 +158,7 @@ case class BloomFilterIndexConfig( } } -object Config { +object HyperSpaceIndexConfigs { // TODO - prints info table about all types of index supported by hyperspace def printAllIndexConfigInfo(): String = { diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala index e0425cbb3..dbfd5b66c 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala @@ -33,7 +33,7 @@ class IndexCollectionManager( extends IndexManager { private val conf: SQLConf = spark.sessionState.conf - override def create(df: DataFrame, indexConfig: IndexConfig): Unit = { + override def create(df: DataFrame, indexConfig: HyperSpaceIndexConfig): Unit = { val hadoopConf = spark.sessionState.newHadoopConf() val indexPath = PathResolver(spark.sessionState.conf, hadoopConf) .getIndexPath(indexConfig.indexName) diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index 7c410244f..3485e80ed 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -16,18 +16,16 @@ package com.microsoft.hyperspace.index -import java.io.FileNotFoundException - -import scala.annotation.tailrec -import scala.collection.mutable.{HashMap, ListBuffer} -import scala.collection.mutable - import com.fasterxml.jackson.annotation.JsonIgnore +import java.io.FileNotFoundException import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types.{DataType, StructType} +import scala.annotation.tailrec +import scala.collection.mutable +import scala.collection.mutable.{HashMap, ListBuffer} import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.actions.Constants @@ -343,59 +341,67 @@ object FileInfo { } } -trait HyperSpaceIndex { - def getKind(): String +object HyperSpaceIndex { + trait IndexType { + def kind: String - def getKindAbbr(): String -} + def kindAbbr: String -trait NonClusteredIndex extends HyperSpaceIndex { - object Kinds extends Enumeration { - type Kinds = String - val covering = "CoveringIndex" - val nonCovering = "NonCoveringIndex" + def properties: Properties.ExposeProperties } -} -// IndexLogEntry-specific CoveringIndex that represents derived dataset. -case class CoveringIndex(properties: CoveringIndex.Properties) extends NonClusteredIndex { - private val kind: String = Kinds.covering - private val kindAbbr: String = "CI" - - override def getKind(): String = { - this.kind + // TODO abstract or trait ? + trait NonClusteredIndex extends IndexType { + object Kinds extends Enumeration { + type Kinds = String + val covering = "CoveringIndex" + val nonCovering = "NonCoveringIndex" + } } - override def getKindAbbr(): String = { - this.kindAbbr - } -} + // IndexLogEntry-specific CoveringIndex that represents derived dataset. + case class CoveringIndex( + coveringProperties: Properties.Covering + ) extends NonClusteredIndex { + + override def kind: String = Kinds.covering -case class BloomFilterIndex(properties: CoveringIndex.BFProperties) extends NonClusteredIndex { - private val kind: String = Kinds.nonCovering - private val kindAbbr: String = "BF" + override def kindAbbr: String = "CI" - override def getKind(): String = { - this.kind + override def properties: Properties.ExposeProperties = coveringProperties } - override def getKindAbbr(): String = { - this.kindAbbr + case class BloomFilterIndex( + bloomProperties: Properties.BloomFilter + ) extends NonClusteredIndex { + override def kind: String = Kinds.nonCovering + + override def kindAbbr: String = "BF" + + override def properties: Properties.ExposeProperties = bloomProperties } -} -object CoveringIndex { - case class Properties(columns: Properties.Columns, - schemaString: String, - numBuckets: Int, - properties: Map[String, String]) + object Properties { + trait ExposeProperties { + def columns: CommonProperties.Columns + + def schemaString: String - case class BFProperties(columns: Properties.Columns, + def properties: Map[String, String] + } + + object CommonProperties { + case class Columns(indexed: Seq[String], included: Seq[String]) + } + + case class Covering(columns: CommonProperties.Columns, schemaString: String, - properties: Map[String, String]) + numBuckets: Int, + properties: Map[String, String]) extends ExposeProperties - object Properties { - case class Columns(indexed: Seq[String], included: Seq[String]) + case class BloomFilter(columns: CommonProperties.Columns, + schemaString: String, + properties: Map[String, String]) extends ExposeProperties } } @@ -471,11 +477,11 @@ case class Source(plan: SparkPlan) // IndexLogEntry that captures index-related information. case class IndexLogEntry( - name: String, - derivedDataset: CoveringIndex, - content: Content, - source: Source, - properties: Map[String, String]) + name: String, + derivedDataset: HyperSpaceIndex.IndexType, + content: Content, + source: Source, + properties: Map[String, String]) extends LogEntry(IndexLogEntry.VERSION) { def schema: StructType = @@ -561,7 +567,13 @@ case class IndexLogEntry( case _ => false } - def numBuckets: Int = derivedDataset.properties.numBuckets + derivedDataset.properties.isInstanceOf + def numBuckets: Int = { + derivedDataset.properties match { + case HyperSpaceIndex.Properties.Covering(_, _, buckets, _) => buckets + case _ => -1 + } + } def config: IndexConfig = IndexConfig(name, indexedColumns, includedColumns) diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexManager.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexManager.scala index 9910d718a..586cab9e7 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexManager.scala @@ -38,7 +38,7 @@ trait IndexManager { * @param indexConfig the configuration of index to be created. * @return Index object that stores information about index created. */ - def create(df: DataFrame, indexConfig: IndexConfig): Unit + def create(df: DataFrame, indexConfig: HyperSpaceIndexConfig): Unit /** * Soft deletes the index with given index name. diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexStatistics.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexStatistics.scala index 654593574..2d836ba41 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexStatistics.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexStatistics.scala @@ -88,7 +88,7 @@ private[hyperspace] object IndexStatistics { entry.derivedDataset.properties.schemaString, indexDirPath(entry), entry.state, - entry.derivedDataset.getKind(), + entry.derivedDataset.kind, entry.hasLineageColumn, entry.content.fileInfos.size, entry.content.fileInfos.foldLeft(0L)(_ + _.size), diff --git a/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/configs/BloomFilterConfig.scala similarity index 78% rename from src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala rename to src/main/scala/com/microsoft/hyperspace/index/configs/BloomFilterConfig.scala index 8ab24a9ed..6dc92081b 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/configs/noncovering/BloomFilterIndexConfig.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/configs/BloomFilterConfig.scala @@ -14,19 +14,18 @@ * limitations under the License. */ -package com.microsoft.hyperspace.index.configs.noncovering +package com.microsoft.hyperspace.index.configs import com.microsoft.hyperspace.index.BloomFilterIndexConfig -import com.microsoft.hyperspace.index.configs.covering.IndexConfig /** - * TODO Defines [[BloomFilterIndexConfig.Builder]] and relevant helper methods for enabling - * builder pattern for [[BloomFilterIndexConfig]]. + * TODO Defines [[BloomFilterConfig.Builder]] and relevant helper methods for enabling + * builder pattern for [[BloomFilterConfig]]. */ -object BloomFilterIndexConfig { +object BloomFilterConfig { /** - * Builder for [[BloomFilterIndexConfig]]. + * Builder for [[BloomFilterConfig]]. */ private[index] class Builder { @@ -37,10 +36,10 @@ object BloomFilterIndexConfig { private[this] var numBits: Long = -1 /** - * Updates index name for [[IndexConfig]]. + * Updates index name for [[CoveringConfig]]. * - * @param indexName index name for the [[BloomFilterIndexConfig]]. - * @return an [[BloomFilterIndexConfig.Builder]] object with updated index name. + * @param indexName index name for the [[BloomFilterConfig]]. + * @return an [[BloomFilterConfig.Builder]] object with updated index name. */ def init(indexName: String, indexedColumn: String): Builder = { if (this.indexName.nonEmpty || this.indexedColumn.nonEmpty) { @@ -107,9 +106,9 @@ object BloomFilterIndexConfig { } /** - * Creates new [[BloomFilterIndexConfig.Builder]] for constructing an [[BloomFilterIndexConfig]]. + * Creates new [[BloomFilterConfig.Builder]] for constructing an [[BloomFilterConfig]]. * - * @return an [[BloomFilterIndexConfig.Builder]] object. + * @return an [[BloomFilterConfig.Builder]] object. */ def builder(): Builder = new Builder } diff --git a/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/configs/CoveringConfig.scala similarity index 64% rename from src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala rename to src/main/scala/com/microsoft/hyperspace/index/configs/CoveringConfig.scala index 838966562..90b2802a5 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/configs/covering/IndexConfig.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/configs/CoveringConfig.scala @@ -14,18 +14,18 @@ * limitations under the License. */ -package com.microsoft.hyperspace.index.configs.covering +package com.microsoft.hyperspace.index.configs import com.microsoft.hyperspace.index.IndexConfig /** - * Defines [[IndexConfig.Builder]] and relevant helper methods for enabling builder pattern for - * [[IndexConfig]]. + * Defines [[CoveringConfig.Builder]] and relevant helper methods for enabling builder + * pattern for [[CoveringConfig]]. */ -object IndexConfig { +object CoveringConfig { /** - * Builder for [[IndexConfig]]. + * Builder for [[CoveringConfig]]. */ class Builder { @@ -34,10 +34,10 @@ object IndexConfig { private[this] var indexName: String = "" /** - * Updates index name for [[IndexConfig]]. + * Updates index name for [[CoveringConfig]]. * - * @param indexName index name for the [[IndexConfig]]. - * @return an [[IndexConfig.Builder]] object with updated index name. + * @param indexName index name for the [[CoveringConfig]]. + * @return an [[CoveringConfig.Builder]] object with updated index name. */ def indexName(indexName: String): Builder = { if (this.indexName.nonEmpty) { @@ -53,13 +53,13 @@ object IndexConfig { } /** - * Updates column names for [[IndexConfig]]. + * Updates column names for [[CoveringConfig]]. * * Note: API signature supports passing one or more argument. * - * @param indexedColumn indexed column for the [[IndexConfig]]. - * @param indexedColumns indexed columns for the [[IndexConfig]]. - * @return an [[IndexConfig.Builder]] object with updated indexed columns. + * @param indexedColumn indexed column for the [[CoveringConfig]]. + * @param indexedColumns indexed columns for the [[CoveringConfig]]. + * @return an [[CoveringConfig.Builder]] object with updated indexed columns. */ def indexBy(indexedColumn: String, indexedColumns: String*): Builder = { if (this.indexedColumns.nonEmpty) { @@ -71,13 +71,13 @@ object IndexConfig { } /** - * Updates included columns for [[IndexConfig]]. + * Updates included columns for [[CoveringConfig]]. * * Note: API signature supports passing one or more argument. * - * @param includedColumn included column for [[IndexConfig]]. - * @param includedColumns included columns for [[IndexConfig]]. - * @return an [[IndexConfig.Builder]] object with updated included columns. + * @param includedColumn included column for [[CoveringConfig]]. + * @param includedColumns included columns for [[CoveringConfig]]. + * @return an [[CoveringConfig.Builder]] object with updated included columns. */ def include(includedColumn: String, includedColumns: String*): Builder = { if (this.includedColumns.nonEmpty) { @@ -90,19 +90,19 @@ object IndexConfig { /** * Creates IndexConfig from supplied index name, indexed columns and included columns - * to [[IndexConfig.Builder]]. + * to [[CoveringConfig.Builder]]. * - * @return an [[IndexConfig]] object. + * @return an [[CoveringConfig]] object. */ def build(): IndexConfig = { - new IndexConfig(indexName, indexedColumns, includedColumns) + IndexConfig(indexName, indexedColumns, includedColumns) } } /** - * Creates new [[IndexConfig.Builder]] for constructing an [[IndexConfig]]. + * Creates new [[CoveringConfig.Builder]] for constructing an [[CoveringConfig]]. * - * @return an [[IndexConfig.Builder]] object. + * @return an [[CoveringConfig.Builder]] object. */ def builder(): Builder = new Builder } diff --git a/src/main/scala/com/microsoft/hyperspace/index/plans/logical/IndexHadoopFsRelation.scala b/src/main/scala/com/microsoft/hyperspace/index/plans/logical/IndexHadoopFsRelation.scala index 1a26dc608..d46b25fb6 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/plans/logical/IndexHadoopFsRelation.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/plans/logical/IndexHadoopFsRelation.scala @@ -42,7 +42,7 @@ class IndexHadoopFsRelation( options)(spark) { val indexPlanStr: String = { - s"Hyperspace(Type: ${index.derivedDataset.getKindAbbr()}, " + + s"Hyperspace(Type: ${index.derivedDataset.kindAbbr}, " + s"Name: ${index.name}, LogVersion: ${index.id})" } override def toString(): String = indexPlanStr diff --git a/src/test/scala/com/microsoft/hyperspace/actions/RefreshActionTest.scala b/src/test/scala/com/microsoft/hyperspace/actions/RefreshActionTest.scala index a978718b5..d737c0e2c 100644 --- a/src/test/scala/com/microsoft/hyperspace/actions/RefreshActionTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/actions/RefreshActionTest.scala @@ -82,9 +82,9 @@ class RefreshActionTest extends SparkFunSuite with SparkInvolvedSuite { val entry = IndexLogEntry( "index1", - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties .Columns(Seq("clicks"), Seq()), "schema", 10, diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexCacheTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexCacheTest.scala index a23403ad8..b7c8fd38e 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexCacheTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexCacheTest.scala @@ -52,9 +52,9 @@ class IndexCacheTest extends SparkFunSuite with SparkInvolvedSuite { val entry = IndexLogEntry( "index1", - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties .Columns(Seq("RGUID"), Seq("Date")), IndexLogEntry.schemaString(schema), 10, diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala index 13dc15a88..fa03d5f7b 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala @@ -48,9 +48,9 @@ class IndexCollectionManagerTest extends SparkFunSuite with SparkInvolvedSuite { val entry = IndexLogEntry( indexPath.toString, - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties .Columns(Seq("RGUID"), Seq("Date")), "", 10, @@ -101,9 +101,9 @@ class IndexCollectionManagerTest extends SparkFunSuite with SparkInvolvedSuite { val entry = IndexLogEntry( str, - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties .Columns(Seq("RGUID"), Seq("Date")), "", 10, diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala index 5cf769f81..575de5746 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexConfigTest.scala @@ -18,10 +18,10 @@ package com.microsoft.hyperspace.index import org.apache.spark.SparkFunSuite -import com.microsoft.hyperspace.index.configs.covering +import com.microsoft.hyperspace.index.configs.CoveringConfig class IndexConfigTest extends SparkFunSuite { - val CoveringIndexConfigBuilder: covering.IndexConfig.type = covering.IndexConfig + val CoveringIndexConfigBuilder: CoveringConfig.type = configs.CoveringConfig test("Empty index name is not allowed.") { diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala index 445c884ca..68663d555 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala @@ -222,9 +222,9 @@ class IndexLogEntryTest extends SparkFunSuite with SQLHelper with BeforeAndAfter val expected = IndexLogEntry( "indexName", - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties .Columns(Seq("col1"), Seq("col2", "col3")), schema.json, 200, diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexLogManagerImplTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexLogManagerImplTest.scala index 789c1de32..ebf347660 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexLogManagerImplTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexLogManagerImplTest.scala @@ -34,9 +34,9 @@ class IndexLogManagerImplTest val testRoot = "src/test/resources/indexLogManagerTests" val sampleIndexLogEntry: IndexLogEntry = IndexLogEntry( "entityName", - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties.Columns(Seq("id"), Seq("name", "school")), + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties.Columns(Seq("id"), Seq("name", "school")), "id INT name STRING school STRING", 100, Map())), diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTest.scala index e45f1f85c..18c85bb10 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTest.scala @@ -786,9 +786,9 @@ class IndexManagerTest extends HyperspaceSuite with SQLHelper { val entry = IndexLogEntry( indexConfig.indexName, - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties .Columns(indexConfig.indexedColumns, indexConfig.includedColumns), IndexLogEntry.schemaString(schema), IndexConstants.INDEX_NUM_BUCKETS_DEFAULT, diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexTest.scala index 2a3f17c48..d9f4d93f2 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexTest.scala @@ -39,9 +39,9 @@ class IndexTest extends SparkFunSuite { val entry = IndexLogEntry( config.indexName, - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties .Columns(config.indexedColumns, config.includedColumns), IndexLogEntry.schemaString(schema), numBuckets, diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala index f49d8ca2d..d41480760 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala @@ -60,9 +60,9 @@ trait HyperspaceRuleSuite extends HyperspaceSuite { val indexLogEntry = IndexLogEntry( name, - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties .Columns(indexCols.map(_.name), includedCols.map(_.name)), IndexLogEntry.schemaString(schemaFromAttributes(indexCols ++ includedCols: _*)), numBuckets, diff --git a/src/test/scala/com/microsoft/hyperspace/util/JsonUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/util/JsonUtilsTest.scala index 686bc6f42..0490eab6a 100644 --- a/src/test/scala/com/microsoft/hyperspace/util/JsonUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/util/JsonUtilsTest.scala @@ -39,9 +39,9 @@ class JsonUtilsTest extends SparkFunSuite { val index = IndexLogEntry( "myIndex", - CoveringIndex( - CoveringIndex.Properties( - CoveringIndex.Properties.Columns(Seq("id"), Seq("name", "school")), + HyperSpaceIndex.CoveringIndex( + HyperSpaceIndex.Properties.Covering( + HyperSpaceIndex.Properties.CommonProperties.Columns(Seq("id"), Seq("name", "school")), IndexLogEntry.schemaString(schema), 10, Map())), From af905e2bc8d40cc492ebfecd334463290b7bd18e Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Tue, 23 Feb 2021 02:31:07 -0800 Subject: [PATCH 07/16] 3 --- .../hyperspace/actions/CreateAction.scala | 2 +- .../hyperspace/actions/CreateActionBase.scala | 75 +++++++++++++++++-- .../index/HyperSpaceIndexConfigs.scala | 2 +- 3 files changed, 69 insertions(+), 10 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index 17b05991d..18ce8d0ec 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -72,7 +72,7 @@ class CreateAction( // TODO: The following should be protected, but RefreshAction is calling CreateAction.op(). // This needs to be refactored to mark this as protected. - final override def op(): Unit = write(spark, df, indexConfig.asInstanceOf[IndexConfig]) + final override def op(): Unit = write(spark, df, indexConfig) final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { // LogEntry instantiation may fail if index config is invalid. Hence the 'Try'. diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index ea0a47315..cdce7c449 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -16,16 +16,17 @@ package com.microsoft.hyperspace.actions -import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} -import org.apache.spark.sql.catalyst.plans.logical.LeafNode -import org.apache.spark.sql.functions.input_file_name - -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} -import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer +import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.sources.FileBasedRelation import com.microsoft.hyperspace.util.{HyperspaceConf, PathUtils, ResolverUtils} +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} +import org.apache.commons.io.output.ByteArrayOutputStream +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.catalyst.plans.logical.LeafNode +import org.apache.spark.sql.functions.{approx_count_distinct, input_file_name} +import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} +import org.apache.spark.util.sketch.BloomFilter /** * CreateActionBase provides functionality to write dataframe as covering index. @@ -120,7 +121,65 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) } } - protected def write(spark: SparkSession, df: DataFrame, indexConfig: IndexConfig): Unit = { + protected def write( + spark: SparkSession, + df: DataFrame, + indexConfig: HyperSpaceIndexConfig): Unit = { + indexConfig match { + case ind: IndexConfig => write(spark, df, ind) + case ind: BloomFilterIndexConfig => write(spark, df, ind) + case _ => HyperspaceException("No write op supported") + } + } + + private def write( + spark: SparkSession, + df: DataFrame, + indexConfig: BloomFilterIndexConfig): Unit = { + val (indexDataFrame, resolvedIndexedColumn, _) = + prepareIndexDataFrame(spark, df, indexConfig) + + require( + resolvedIndexedColumn.size == 1, + "Resolved indexed columns for Bloom Filter can only be 1") + + val resolvedNumBits = indexConfig.numBits match { + case -1 => BloomFilter.create(indexConfig.expectedNumItems).bitSize() + case _ => indexConfig.numBits + } + + val expectedGlobalItems: Long = + df.agg(approx_count_distinct(resolvedIndexedColumn.head)).collect()(0)(0).asInstanceOf[Long] + val globalBF = BloomFilter.create(expectedGlobalItems, resolvedNumBits) + assert(globalBF.isCompatible(BloomFilter.create(indexConfig.expectedNumItems, resolvedNumBits))) + + // Begin has this op as relation is created there + import spark.implicits._ + val relations = getRelation(spark, df).createRelationMetadata(fileIdTracker) + val resultColSeq = Seq("path", "BFData", "Active") + val result = Seq.empty[(String, String, String)].toDF(resultColSeq: _*) + relations.rootPaths.foreach(path => { + val bfByteStream = new ByteArrayOutputStream() + val localBF = spark.read + .schema(df.schema) + .format(relations.fileFormat) + .options(relations.options) + .load(path) + .select(resolvedIndexedColumn.head) + .stat + .bloomFilter( + resolvedIndexedColumn.head, + indexConfig.expectedNumItems, + resolvedNumBits) + localBF.writeTo(bfByteStream) + bfByteStream.close() + globalBF.mergeInPlace(localBF) + result.union( + Seq(path, bfByteStream.toByteArray.map(_.toChar).mkString, "1").toDF(resultColSeq: _*)) + }) + } + + private def write(spark: SparkSession, df: DataFrame, indexConfig: IndexConfig): Unit = { val numBuckets = numBucketsForIndex(spark) val (indexDataFrame, resolvedIndexedColumns, _) = diff --git a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfigs.scala b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfigs.scala index f8de8b19f..c60ff3093 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfigs.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfigs.scala @@ -112,7 +112,7 @@ case class BloomFilterIndexConfig( indexName: String, indexedColumn: String, expectedNumItems: Long, - fpp: Double = -1, + fpp: Double = 0.03, // DEFAULT FPP in Apache Spark Bloom Filter Implementation numBits: Long = -1) extends NonCoveringIndexConfig { if (indexName.isEmpty || indexedColumn.isEmpty || expectedNumItems < 1) { From 9b4af0e072c930545cdcf211f6d97b2b2aeb0cd1 Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Tue, 23 Feb 2021 13:41:41 -0800 Subject: [PATCH 08/16] now multiple index in core supported --- .../hyperspace/actions/CreateActionBase.scala | 13 ++++++++++--- .../hyperspace/index/IndexLogEntry.scala | 16 ++++++++++++++-- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index cdce7c449..09ef8d35f 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -136,7 +136,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) spark: SparkSession, df: DataFrame, indexConfig: BloomFilterIndexConfig): Unit = { - val (indexDataFrame, resolvedIndexedColumn, _) = + val (_, resolvedIndexedColumn, _) = prepareIndexDataFrame(spark, df, indexConfig) require( @@ -153,10 +153,11 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) val globalBF = BloomFilter.create(expectedGlobalItems, resolvedNumBits) assert(globalBF.isCompatible(BloomFilter.create(indexConfig.expectedNumItems, resolvedNumBits))) - // Begin has this op as relation is created there + // TODO Begin has this op as relation is created there + // TODO Maybe use lineage to make file smaller import spark.implicits._ val relations = getRelation(spark, df).createRelationMetadata(fileIdTracker) - val resultColSeq = Seq("path", "BFData", "Active") + val resultColSeq = Seq("Path", "BFData", "Active") val result = Seq.empty[(String, String, String)].toDF(resultColSeq: _*) relations.rootPaths.foreach(path => { val bfByteStream = new ByteArrayOutputStream() @@ -177,6 +178,12 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) result.union( Seq(path, bfByteStream.toByteArray.map(_.toChar).mkString, "1").toDF(resultColSeq: _*)) }) + val gbfByteStream = new ByteArrayOutputStream() + globalBF.writeTo(gbfByteStream) + gbfByteStream.close() + result.union( + Seq("GLOBAL", gbfByteStream.toByteArray.map(_.toChar).mkString, "1").toDF(resultColSeq: _*)) + result.write.parquet(new Path(indexDataPath, "bf.parquet").toString) } private def write(spark: SparkSession, df: DataFrame, indexConfig: IndexConfig): Unit = { diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index 3485e80ed..a52a4b4e7 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -16,17 +16,19 @@ package com.microsoft.hyperspace.index -import com.fasterxml.jackson.annotation.JsonIgnore +import com.fasterxml.jackson.annotation.JsonSubTypes.Type +import com.fasterxml.jackson.annotation.{JsonIgnore, JsonSubTypes, JsonTypeInfo} + import java.io.FileNotFoundException import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types.{DataType, StructType} + import scala.annotation.tailrec import scala.collection.mutable import scala.collection.mutable.{HashMap, ListBuffer} - import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.util.PathUtils @@ -342,6 +344,16 @@ object FileInfo { } object HyperSpaceIndex { + + @JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + include = JsonTypeInfo.As.PROPERTY, + property = "type" + ) + @JsonSubTypes(Array( + new Type(value = classOf[CoveringIndex], name = "coveringIndex"), + new Type(value = classOf[BloomFilterIndex], name = "bloomFilterIndex") + )) trait IndexType { def kind: String From 8b63a9a4061ec43f51f74f63cf8ff94f712f77cc Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Tue, 23 Feb 2021 22:18:54 -0800 Subject: [PATCH 09/16] lineage test fixed --- .../microsoft/hyperspace/actions/CreateActionBase.scala | 4 ++-- .../com/microsoft/hyperspace/index/IndexLogEntry.scala | 9 ++++----- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 09ef8d35f..64f13e8a0 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -293,9 +293,9 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) .select( allIndexColumns.head, allIndexColumns.tail :+ IndexConstants.DATA_FILE_NAME_ID: _*) + } else { + df.select(providedIndexColumns.head, providedIndexColumns.tail: _*) } - - df.select(providedIndexColumns.head, providedIndexColumns.tail: _*) } private def prepareIndexDataFrame( diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index a52a4b4e7..9066d7011 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -18,20 +18,19 @@ package com.microsoft.hyperspace.index import com.fasterxml.jackson.annotation.JsonSubTypes.Type import com.fasterxml.jackson.annotation.{JsonIgnore, JsonSubTypes, JsonTypeInfo} - -import java.io.FileNotFoundException +import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.actions.Constants +import com.microsoft.hyperspace.util.PathUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types.{DataType, StructType} +import java.io.FileNotFoundException import scala.annotation.tailrec import scala.collection.mutable import scala.collection.mutable.{HashMap, ListBuffer} -import com.microsoft.hyperspace.HyperspaceException -import com.microsoft.hyperspace.actions.Constants -import com.microsoft.hyperspace.util.PathUtils // IndexLogEntry-specific fingerprint to be temporarily used where fingerprint is not defined. case class NoOpFingerprint() { From de9b10d0eb56847def47b6076265f9472d50435b Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Wed, 24 Feb 2021 14:09:01 -0800 Subject: [PATCH 10/16] bf data created and stored --- .../hyperspace/actions/CreateAction.scala | 32 ++++++--- .../hyperspace/actions/CreateActionBase.scala | 18 ++--- ...rSpaceIndexConfigs.scala => Configs.scala} | 30 +------- .../hyperspace/index/HyperSpaceIndex.scala | 69 ++++++++++++++++++ .../index/HyperSpaceIndexConfig.scala | 48 +++++++++++++ .../hyperspace/index/IndexLogEntry.scala | 72 ------------------- .../telemetry/HyperspaceEvent.scala | 12 ++-- .../com/microsoft/hyperspace/SampleData.scala | 57 +++++++++++++++ .../hyperspace/index/CreateIndexTest.scala | 18 ++++- 9 files changed, 228 insertions(+), 128 deletions(-) rename src/main/scala/com/microsoft/hyperspace/index/{HyperSpaceIndexConfigs.scala => Configs.scala} (88%) create mode 100644 src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndex.scala create mode 100644 src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index 18ce8d0ec..7f1369a80 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -16,15 +16,15 @@ package com.microsoft.hyperspace.actions -import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.types.StructType -import scala.util.Try - -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, CREATING, DOESNOTEXIST} import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.telemetry.{AppInfo, CreateActionEvent, HyperspaceEvent} import com.microsoft.hyperspace.util.ResolverUtils +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, SparkSession} + +import scala.util.Try class CreateAction( spark: SparkSession, @@ -49,7 +49,7 @@ class CreateAction( } // schema validity checks - if (!isValidIndexSchema(indexConfig.asInstanceOf[IndexConfig], df.schema)) { + if (!isValidIndexSchema(indexConfig, df.schema)) { throw HyperspaceException("Index config is not applicable to dataframe schema.") } @@ -63,11 +63,20 @@ class CreateAction( } } - private def isValidIndexSchema(config: IndexConfig, schema: StructType): Boolean = { + private def isValidIndexSchema(config: HyperSpaceIndexConfig, schema: StructType): Boolean = { // Resolve index config columns from available column names present in the schema. - ResolverUtils - .resolve(spark, config.indexedColumns ++ config.includedColumns, schema.fieldNames) - .isDefined + config match { + case indexConfig: IndexConfig => + ResolverUtils + .resolve( + spark, + indexConfig.indexedColumns ++ indexConfig.includedColumns, + schema.fieldNames) + .isDefined + case indexConfig: BloomFilterIndexConfig => + ResolverUtils.resolve(spark, indexConfig.indexedColumn, schema.fieldNames).isDefined + } + } // TODO: The following should be protected, but RefreshAction is calling CreateAction.op(). @@ -76,10 +85,11 @@ class CreateAction( final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { // LogEntry instantiation may fail if index config is invalid. Hence the 'Try'. + val index = Try(logEntry.asInstanceOf[IndexLogEntry]).toOption CreateActionEvent( appInfo, - indexConfig.asInstanceOf[IndexConfig], + IndexConfigBundle(indexConfig), index, df.queryExecution.logical.toString, message) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 64f13e8a0..ba74b2ccd 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -151,7 +151,8 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) val expectedGlobalItems: Long = df.agg(approx_count_distinct(resolvedIndexedColumn.head)).collect()(0)(0).asInstanceOf[Long] val globalBF = BloomFilter.create(expectedGlobalItems, resolvedNumBits) - assert(globalBF.isCompatible(BloomFilter.create(indexConfig.expectedNumItems, resolvedNumBits))) + assert( + globalBF.isCompatible(BloomFilter.create(indexConfig.expectedNumItems, resolvedNumBits))) // TODO Begin has this op as relation is created there // TODO Maybe use lineage to make file smaller @@ -168,21 +169,20 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) .load(path) .select(resolvedIndexedColumn.head) .stat - .bloomFilter( - resolvedIndexedColumn.head, - indexConfig.expectedNumItems, - resolvedNumBits) + .bloomFilter(resolvedIndexedColumn.head, indexConfig.expectedNumItems, resolvedNumBits) localBF.writeTo(bfByteStream) bfByteStream.close() globalBF.mergeInPlace(localBF) - result.union( - Seq(path, bfByteStream.toByteArray.map(_.toChar).mkString, "1").toDF(resultColSeq: _*)) + val bfBinaryCharStream = bfByteStream.toByteArray.map(_.toChar).mkString + val bfRecord = Seq((path, bfBinaryCharStream, "1")).toDF(resultColSeq: _*) + result.union(bfRecord) }) val gbfByteStream = new ByteArrayOutputStream() globalBF.writeTo(gbfByteStream) gbfByteStream.close() - result.union( - Seq("GLOBAL", gbfByteStream.toByteArray.map(_.toChar).mkString, "1").toDF(resultColSeq: _*)) + val globalBFRecord = Seq(("GLOBAL", gbfByteStream.toByteArray.map(_.toChar).mkString, "1")) + .toDF(resultColSeq: _*) + result.union(globalBFRecord) result.write.parquet(new Path(indexDataPath, "bf.parquet").toString) } diff --git a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfigs.scala b/src/main/scala/com/microsoft/hyperspace/index/Configs.scala similarity index 88% rename from src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfigs.scala rename to src/main/scala/com/microsoft/hyperspace/index/Configs.scala index c60ff3093..d1dc4ab09 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfigs.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/Configs.scala @@ -20,34 +20,6 @@ import java.util.Locale import com.microsoft.hyperspace.index.configs.CoveringConfig -/** - * All index supported in Hyperspace whose user facing config needs to be defined needs - * to extend [[HyperSpaceIndexConfig]] trait. - */ -sealed trait HyperSpaceIndexConfig { - val indexName: String - - def equals(obj: Any): Boolean - - def hashCode(): Int - - def toString: String -} - -trait CoveringIndexConfig extends HyperSpaceIndexConfig { - /* - * Columns from which index are created. - */ - val indexedColumns: Seq[String] - - /* - * Columns to be included with the indexed columns. - */ - val includedColumns: Seq[String] -} - -trait NonCoveringIndexConfig extends HyperSpaceIndexConfig {} - /** * IndexConfig specifies the configuration of an index. * Associated Builder [[CoveringConfig.builder()]] @@ -158,7 +130,7 @@ case class BloomFilterIndexConfig( } } -object HyperSpaceIndexConfigs { +object Configs { // TODO - prints info table about all types of index supported by hyperspace def printAllIndexConfigInfo(): String = { diff --git a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndex.scala b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndex.scala new file mode 100644 index 000000000..d3e003ad0 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndex.scala @@ -0,0 +1,69 @@ +package com.microsoft.hyperspace.index + +import com.fasterxml.jackson.annotation.{JsonSubTypes, JsonTypeInfo} +import com.fasterxml.jackson.annotation.JsonSubTypes.Type + +object HyperSpaceIndex { + + @JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + include = JsonTypeInfo.As.PROPERTY, + property = "type" + ) + @JsonSubTypes(Array( + new Type(value = classOf[CoveringIndex], name = "coveringIndex"), + new Type(value = classOf[BloomFilterIndex], name = "bloomFilterIndex") + )) + trait IndexType { + def kind: String + + def kindAbbr: String + + def properties: Properties.ExposeProperties + } + + // IndexLogEntry-specific CoveringIndex that represents derived dataset. + case class CoveringIndex( + coveringProperties: Properties.Covering + ) extends IndexType { + + override def kind: String = Kinds.covering + + override def kindAbbr: String = "CI" + + override def properties: Properties.ExposeProperties = coveringProperties + } + + case class BloomFilterIndex( + bloomProperties: Properties.BloomFilter + ) extends IndexType { + override def kind: String = Kinds.nonCovering + + override def kindAbbr: String = "BF" + + override def properties: Properties.ExposeProperties = bloomProperties + } + + object Properties { + trait ExposeProperties { + def columns: CommonProperties.Columns + + def schemaString: String + + def properties: Map[String, String] + } + + object CommonProperties { + case class Columns(indexed: Seq[String], included: Seq[String]) + } + + case class Covering(columns: CommonProperties.Columns, + schemaString: String, + numBuckets: Int, + properties: Map[String, String]) extends ExposeProperties + + case class BloomFilter(columns: CommonProperties.Columns, + schemaString: String, + properties: Map[String, String]) extends ExposeProperties + } +} \ No newline at end of file diff --git a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala new file mode 100644 index 000000000..28f4d5881 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala @@ -0,0 +1,48 @@ +package com.microsoft.hyperspace.index + +/** + * All index supported in Hyperspace whose user facing config needs to be defined needs + * to extend [[HyperSpaceIndexConfig]] trait. + */ +sealed trait HyperSpaceIndexConfig { + def indexName: String + + def kind: String +} + +object Kinds extends Enumeration { + type Kinds = String + val clustered = "Clustered" + val nonClustered: Kinds = "NonClustered" + val covering: Kinds = nonClustered + "-CoveringIndex" + val nonCovering: Kinds = nonClustered + "-NonCoveringIndex" + + def identifyKind(provided: Kinds): Kinds = { + if (provided.contains(clustered)) { + clustered + } else { + nonClustered + } + } +} + +abstract class CoveringIndexConfig extends HyperSpaceIndexConfig { + def kind: String = Kinds.covering + + /* + * Columns from which index are created. + */ + def indexedColumns: Seq[String] + + /* + * Columns to be included with the indexed columns. + */ + def includedColumns: Seq[String] +} + +abstract class NonCoveringIndexConfig extends HyperSpaceIndexConfig { + def kind: String = Kinds.nonCovering +} + +case class IndexConfigBundle(config: HyperSpaceIndexConfig) { +} diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index 9066d7011..ab5baf18b 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -342,79 +342,7 @@ object FileInfo { } } -object HyperSpaceIndex { - - @JsonTypeInfo( - use = JsonTypeInfo.Id.NAME, - include = JsonTypeInfo.As.PROPERTY, - property = "type" - ) - @JsonSubTypes(Array( - new Type(value = classOf[CoveringIndex], name = "coveringIndex"), - new Type(value = classOf[BloomFilterIndex], name = "bloomFilterIndex") - )) - trait IndexType { - def kind: String - - def kindAbbr: String - - def properties: Properties.ExposeProperties - } - - // TODO abstract or trait ? - trait NonClusteredIndex extends IndexType { - object Kinds extends Enumeration { - type Kinds = String - val covering = "CoveringIndex" - val nonCovering = "NonCoveringIndex" - } - } - - // IndexLogEntry-specific CoveringIndex that represents derived dataset. - case class CoveringIndex( - coveringProperties: Properties.Covering - ) extends NonClusteredIndex { - - override def kind: String = Kinds.covering - - override def kindAbbr: String = "CI" - - override def properties: Properties.ExposeProperties = coveringProperties - } - - case class BloomFilterIndex( - bloomProperties: Properties.BloomFilter - ) extends NonClusteredIndex { - override def kind: String = Kinds.nonCovering - - override def kindAbbr: String = "BF" - - override def properties: Properties.ExposeProperties = bloomProperties - } - - object Properties { - trait ExposeProperties { - def columns: CommonProperties.Columns - - def schemaString: String - - def properties: Map[String, String] - } - object CommonProperties { - case class Columns(indexed: Seq[String], included: Seq[String]) - } - - case class Covering(columns: CommonProperties.Columns, - schemaString: String, - numBuckets: Int, - properties: Map[String, String]) extends ExposeProperties - - case class BloomFilter(columns: CommonProperties.Columns, - schemaString: String, - properties: Map[String, String]) extends ExposeProperties - } -} // IndexLogEntry-specific Signature that stores the signature provider and value. case class Signature(provider: String, value: String) diff --git a/src/main/scala/com/microsoft/hyperspace/telemetry/HyperspaceEvent.scala b/src/main/scala/com/microsoft/hyperspace/telemetry/HyperspaceEvent.scala index 8957a1db8..ea032a4d7 100644 --- a/src/main/scala/com/microsoft/hyperspace/telemetry/HyperspaceEvent.scala +++ b/src/main/scala/com/microsoft/hyperspace/telemetry/HyperspaceEvent.scala @@ -16,7 +16,7 @@ package com.microsoft.hyperspace.telemetry -import com.microsoft.hyperspace.index.{IndexConfig, IndexLogEntry} +import com.microsoft.hyperspace.index.{IndexConfig, IndexConfigBundle, IndexLogEntry} /** * Class for common app info. @@ -47,11 +47,11 @@ trait HyperspaceIndexCRUDEvent extends HyperspaceEvent * @param message Message about event. */ case class CreateActionEvent( - appInfo: AppInfo, - indexConfig: IndexConfig, - index: Option[IndexLogEntry], - originalPlan: String, - message: String) + appInfo: AppInfo, + indexConfig: IndexConfigBundle, + index: Option[IndexLogEntry], + originalPlan: String, + message: String) extends HyperspaceIndexCRUDEvent /** diff --git a/src/test/scala/com/microsoft/hyperspace/SampleData.scala b/src/test/scala/com/microsoft/hyperspace/SampleData.scala index 1b671a08b..406ce1e2f 100644 --- a/src/test/scala/com/microsoft/hyperspace/SampleData.scala +++ b/src/test/scala/com/microsoft/hyperspace/SampleData.scala @@ -34,6 +34,47 @@ object SampleData { ("2019-10-03", "ff60e4838b92421eafc3e6ee59a9e9f1", "miperro", 2, 2000), ("2019-10-03", "187696fe0a6a40cc9516bc6e47c70bc1", "facebook", 4, 3000)) + val testDataMarvel = Seq( + (1941, "Captain America", "Shield", "Avengers", "Marvel"), + (1966, "Black Panther", "Claws", "Avengers", "Marvel"), + (1973, "Blade", "Sword", "Vampire", "Marvel"), + (1974, "Wolverine", "Claws", "XMen", "Marvel"), + (1964, "Scarlet Witch", "Telepathy", "Avengers", "Marvel"), + (1968, "Vision", "Mind Stone", "Avengers", "Marvel"), + (1972, "Ghost Rider", "Soul", "SHIELD", "Marvel"), + (1962, "Hulk", "Gamma Radiation Strength", "Avengers", "Marvel"), + (1963, "Spider Man", "Spider Web", "Avengers", "Marvel"), + (1948, "Loki", "Magic Sword", "Avengers", "Marvel"), + (1966, "Silver Surfer", "Board", "Fantastic Four", "Marvel"), + (1968, "Doctor Strange", "Eye of Agamotto", "Avengers", "Marvel"), + (1963, "Cyclops", "Optic eyes", "XMen", "Marvel"), + (1963, "Beast", "Strength", "XMen", "Marvel"), + (1963, "Iceman", "Ice", "XMen", "Marvel"), + (1963, "Jean Grey", "Telepathy", "XMen", "Marvel"), + (1963, "Nick Fury", "One eye", "SHIELD", "Marvel"), + (1963, "Deadpool", "Katanas", "XMen", "Marvel"), + (1963, "Iron Man", "Suit", "Avengers", "Marvel"), + (1963, "ProfessorX", "Telepathy", "XMen", "Marvel"), + (1973, "Thanos", "Infinity Gauntlet", "Avengers", "Marvel"), + (1962, "Thor", "Hammer", "Avengers", "Marvel"), + (1962, "Ant Man", "Pym Particles", "Avengers", "Marvel"), + (1963, "Magneto", "Magnetic Forces", "XMen", "Marvel"), + (1976, "Star Lord", "Half Human Half Celestial", "Guardians Of Galaxy", "Marvel"), + ) + + val testDataDC = Seq( + (1941, "Aquaman", "Water", "Justice League", "DC"), + (1941, "Wonder Woman", "Lasso", "Justice League", "DC"), + (1940, "Shazam", "Magic", "Marvel Family", "DC"), + (1938, "Superman", "Superhuman Strength", "Justice League", "DC"), + (1956, "Flash", "Speed", "Justice League", "DC"), + (1959, "Green Lantern", "Ring", "Justice League", "DC"), + (1939, "Batman", "None", "Justice League", "DC"), + (1940, "Joker", "Chemist Engineer", "Injustice League", "DC"), + (1992, "Harley Quinn", "Gymnast", "Suicide Squad", "DC"), + (1950, "Deadshot", "Marksman", "Suicide Squad", "DC") + ) + def save( spark: SparkSession, path: String, @@ -48,4 +89,20 @@ object SampleData { df.write.parquet(path) } } + + def saveComics( + spark: SparkSession, + data: Seq[(Int, String, String, String, String)], + path: String, + columns: Seq[String], + partitionColumns: Option[Seq[String]] = None): Unit = { + import spark.implicits._ + val df = data.toDF(columns: _*) + partitionColumns match { + case Some(pcs) => + df.write.partitionBy(pcs: _*).parquet(path) + case None => + df.write.parquet(path) + } + } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala index 9a5914d5e..e7ad875eb 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala @@ -31,13 +31,18 @@ class CreateIndexTest extends HyperspaceSuite with SQLHelper { private val testDir = "src/test/resources/createIndexTests/" private val nonPartitionedDataPath = testDir + "sampleparquet" private val partitionedDataPath = testDir + "samplepartitionedparquet" + private val comicDataDir = testDir + "comics/" + private val dcDataPath = comicDataDir + "dc" + private val marvelDataPath = comicDataDir + "marvel" private val partitionKeys = Seq("Date", "Query") private val indexConfig1 = IndexConfig("index1", Seq("RGUID"), Seq("Date")) private val indexConfig2 = IndexConfig("index2", Seq("Query"), Seq("imprs")) private val indexConfig3 = IndexConfig("index3", Seq("imprs"), Seq("clicks")) private val indexConfig4 = IndexConfig("index4", Seq("Date", "Query"), Seq("clicks")) + private val bloomIndexConfig = BloomFilterIndexConfig("indexBF1", "Affiliation", 10) private var nonPartitionedDataDF: DataFrame = _ private var partitionedDataDF: DataFrame = _ + private var comicDataDF: DataFrame = _ private var hyperspace: Hyperspace = _ override def beforeAll(): Unit = { @@ -47,6 +52,8 @@ class CreateIndexTest extends HyperspaceSuite with SQLHelper { FileUtils.delete(new Path(testDir), isRecursive = true) val dataColumns = Seq("Date", "RGUID", "Query", "imprs", "clicks") + val comicDataColumns = Seq("Year", "Name", "Power", "Affiliation", "Universe") + // save test data non-partitioned. SampleData.save(spark, nonPartitionedDataPath, dataColumns) nonPartitionedDataDF = spark.read.parquet(nonPartitionedDataPath) @@ -54,6 +61,11 @@ class CreateIndexTest extends HyperspaceSuite with SQLHelper { // save test data partitioned. SampleData.save(spark, partitionedDataPath, dataColumns, Some(partitionKeys)) partitionedDataDF = spark.read.parquet(partitionedDataPath) + + // save test data comic. + SampleData.saveComics(spark, SampleData.testDataMarvel, marvelDataPath, comicDataColumns) + SampleData.saveComics(spark, SampleData.testDataDC, dcDataPath, comicDataColumns) + comicDataDF = spark.read.parquet(marvelDataPath, dcDataPath) } override def afterAll(): Unit = { @@ -65,12 +77,16 @@ class CreateIndexTest extends HyperspaceSuite with SQLHelper { FileUtils.delete(systemPath) } - test("Creating one index.") { + test("Creating one covering index.") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig1) val count = hyperspace.indexes.where(s"name = '${indexConfig1.indexName}' ").count assert(count == 1) } + test("Creating one bloom filter index.") { + hyperspace.createIndex(comicDataDF, bloomIndexConfig) + } + test("Creating index with existing index name fails.") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig1) val exception = intercept[HyperspaceException] { From b4a897b527d4579919ccb80e83d2d4e423f0110a Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Wed, 24 Feb 2021 14:33:50 -0800 Subject: [PATCH 11/16] build failures --- .../hyperspace/actions/CreateAction.scala | 10 +++++----- .../hyperspace/actions/CreateActionBase.scala | 13 +++++++------ .../hyperspace/index/HyperSpaceIndex.scala | 18 +++++++++++++++++- .../index/HyperSpaceIndexConfig.scala | 16 ++++++++++++++++ .../hyperspace/index/IndexLogEntry.scala | 13 ++++++------- .../hyperspace/index/CreateIndexTest.scala | 2 ++ 6 files changed, 53 insertions(+), 19 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index 7f1369a80..7ee4d8f2b 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -16,15 +16,15 @@ package com.microsoft.hyperspace.actions +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.types.StructType +import scala.util.Try + +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, CREATING, DOESNOTEXIST} import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.telemetry.{AppInfo, CreateActionEvent, HyperspaceEvent} import com.microsoft.hyperspace.util.ResolverUtils -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{DataFrame, SparkSession} - -import scala.util.Try class CreateAction( spark: SparkSession, diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index ba74b2ccd..bfd74103b 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -16,18 +16,19 @@ package com.microsoft.hyperspace.actions -import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer -import com.microsoft.hyperspace.index._ -import com.microsoft.hyperspace.index.sources.FileBasedRelation -import com.microsoft.hyperspace.util.{HyperspaceConf, PathUtils, ResolverUtils} -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} import org.apache.commons.io.output.ByteArrayOutputStream import org.apache.hadoop.fs.Path +import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.functions.{approx_count_distinct, input_file_name} -import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} import org.apache.spark.util.sketch.BloomFilter +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} +import com.microsoft.hyperspace.index._ +import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer +import com.microsoft.hyperspace.index.sources.FileBasedRelation +import com.microsoft.hyperspace.util.{HyperspaceConf, PathUtils, ResolverUtils} + /** * CreateActionBase provides functionality to write dataframe as covering index. */ diff --git a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndex.scala b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndex.scala index d3e003ad0..67b775667 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndex.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndex.scala @@ -1,3 +1,19 @@ +/* + * Copyright (2020) The Hyperspace Project Authors. + * + * Licensed 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 com.microsoft.hyperspace.index import com.fasterxml.jackson.annotation.{JsonSubTypes, JsonTypeInfo} @@ -66,4 +82,4 @@ object HyperSpaceIndex { schemaString: String, properties: Map[String, String]) extends ExposeProperties } -} \ No newline at end of file +} diff --git a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala index 28f4d5881..4a59a2360 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala @@ -1,3 +1,19 @@ +/* + * Copyright (2020) The Hyperspace Project Authors. + * + * Licensed 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 com.microsoft.hyperspace.index /** diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index ab5baf18b..4397d8958 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -16,22 +16,21 @@ package com.microsoft.hyperspace.index -import com.fasterxml.jackson.annotation.JsonSubTypes.Type -import com.fasterxml.jackson.annotation.{JsonIgnore, JsonSubTypes, JsonTypeInfo} -import com.microsoft.hyperspace.HyperspaceException -import com.microsoft.hyperspace.actions.Constants -import com.microsoft.hyperspace.util.PathUtils +import com.fasterxml.jackson.annotation.JsonIgnore +import java.io.FileNotFoundException import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types.{DataType, StructType} - -import java.io.FileNotFoundException import scala.annotation.tailrec import scala.collection.mutable import scala.collection.mutable.{HashMap, ListBuffer} +import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.actions.Constants +import com.microsoft.hyperspace.util.PathUtils + // IndexLogEntry-specific fingerprint to be temporarily used where fingerprint is not defined. case class NoOpFingerprint() { val kind: String = "NoOp" diff --git a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala index e7ad875eb..95bd2ee03 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala @@ -85,6 +85,8 @@ class CreateIndexTest extends HyperspaceSuite with SQLHelper { test("Creating one bloom filter index.") { hyperspace.createIndex(comicDataDF, bloomIndexConfig) + val count = hyperspace.indexes.where(s"name = '${bloomIndexConfig.indexName}' ").count + assert(count == 1) } test("Creating index with existing index name fails.") { From aab7725e9d88d6d66c173fe8c21adce81cb3a2c1 Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Wed, 24 Feb 2021 15:23:25 -0800 Subject: [PATCH 12/16] code --- .../hyperspace/actions/CreateActionBase.scala | 2 +- .../hyperspace/index/HyperSpaceIndexConfig.scala | 8 ++++---- .../scala/com/microsoft/hyperspace/SampleData.scala | 13 ++++++------- 3 files changed, 11 insertions(+), 12 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index bfd74103b..d9f0c406f 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -88,7 +88,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) IndexLogEntry.schemaString(indexDataFrame.schema), numBuckets, indexProperties)) - case _: HyperSpaceIndexConfig => + case _: BloomFilterIndexConfig => HyperSpaceIndex.BloomFilterIndex( HyperSpaceIndex.Properties.BloomFilter( HyperSpaceIndex.Properties.CommonProperties diff --git a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala index 4a59a2360..5110a92f0 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/HyperSpaceIndexConfig.scala @@ -33,11 +33,11 @@ object Kinds extends Enumeration { val covering: Kinds = nonClustered + "-CoveringIndex" val nonCovering: Kinds = nonClustered + "-NonCoveringIndex" - def identifyKind(provided: Kinds): Kinds = { - if (provided.contains(clustered)) { - clustered - } else { + def identifyBaseKind(provided: Kinds): Kinds = { + if (provided.contains(nonClustered)) { nonClustered + } else { + clustered } } } diff --git a/src/test/scala/com/microsoft/hyperspace/SampleData.scala b/src/test/scala/com/microsoft/hyperspace/SampleData.scala index 406ce1e2f..e04e3d377 100644 --- a/src/test/scala/com/microsoft/hyperspace/SampleData.scala +++ b/src/test/scala/com/microsoft/hyperspace/SampleData.scala @@ -72,8 +72,7 @@ object SampleData { (1939, "Batman", "None", "Justice League", "DC"), (1940, "Joker", "Chemist Engineer", "Injustice League", "DC"), (1992, "Harley Quinn", "Gymnast", "Suicide Squad", "DC"), - (1950, "Deadshot", "Marksman", "Suicide Squad", "DC") - ) + (1950, "Deadshot", "Marksman", "Suicide Squad", "DC")) def save( spark: SparkSession, @@ -91,11 +90,11 @@ object SampleData { } def saveComics( - spark: SparkSession, - data: Seq[(Int, String, String, String, String)], - path: String, - columns: Seq[String], - partitionColumns: Option[Seq[String]] = None): Unit = { + spark: SparkSession, + data: Seq[(Int, String, String, String, String)], + path: String, + columns: Seq[String], + partitionColumns: Option[Seq[String]] = None): Unit = { import spark.implicits._ val df = data.toDF(columns: _*) partitionColumns match { From 18c5ce797e6b629c3f69a0133eda0afd7fa8ca90 Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Sun, 21 Mar 2021 10:35:16 -0700 Subject: [PATCH 13/16] udf bloom --- .../hyperspace/actions/CreateActionBase.scala | 42 ++++++++++--------- 1 file changed, 23 insertions(+), 19 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index d9f0c406f..31a4f12a3 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -20,9 +20,8 @@ import org.apache.commons.io.output.ByteArrayOutputStream import org.apache.hadoop.fs.Path import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.LeafNode -import org.apache.spark.sql.functions.{approx_count_distinct, input_file_name} +import org.apache.spark.sql.functions.{approx_count_distinct, col, input_file_name, udf} import org.apache.spark.util.sketch.BloomFilter - import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer @@ -133,6 +132,8 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) } } + + private def write( spark: SparkSession, df: DataFrame, @@ -149,6 +150,16 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) case _ => indexConfig.numBits } + def cleanCountry = (country: String) => { + val allUSA = Seq("US", "USa", "USA", "United states", "United states of America") + if (allUSA.contains(country)) { + "USA" + } + else { + "unknown" + } + } + val expectedGlobalItems: Long = df.agg(approx_count_distinct(resolvedIndexedColumn.head)).collect()(0)(0).asInstanceOf[Long] val globalBF = BloomFilter.create(expectedGlobalItems, resolvedNumBits) @@ -157,14 +168,10 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) // TODO Begin has this op as relation is created there // TODO Maybe use lineage to make file smaller - import spark.implicits._ val relations = getRelation(spark, df).createRelationMetadata(fileIdTracker) - val resultColSeq = Seq("Path", "BFData", "Active") - val result = Seq.empty[(String, String, String)].toDF(resultColSeq: _*) - relations.rootPaths.foreach(path => { + val bloomFilterUDF = udf((path: String) => { val bfByteStream = new ByteArrayOutputStream() - val localBF = spark.read - .schema(df.schema) + val localBF = spark.read.schema(df.schema) .format(relations.fileFormat) .options(relations.options) .load(path) @@ -173,18 +180,15 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) .bloomFilter(resolvedIndexedColumn.head, indexConfig.expectedNumItems, resolvedNumBits) localBF.writeTo(bfByteStream) bfByteStream.close() - globalBF.mergeInPlace(localBF) - val bfBinaryCharStream = bfByteStream.toByteArray.map(_.toChar).mkString - val bfRecord = Seq((path, bfBinaryCharStream, "1")).toDF(resultColSeq: _*) - result.union(bfRecord) + bfByteStream.toByteArray.map(_.toChar).mkString }) - val gbfByteStream = new ByteArrayOutputStream() - globalBF.writeTo(gbfByteStream) - gbfByteStream.close() - val globalBFRecord = Seq(("GLOBAL", gbfByteStream.toByteArray.map(_.toChar).mkString, "1")) - .toDF(resultColSeq: _*) - result.union(globalBFRecord) - result.write.parquet(new Path(indexDataPath, "bf.parquet").toString) + + val bloomFilterDF = spark.createDataFrame( + relations.rootPaths.map(p => Tuple1(p)) + ).toDF("FileName") + val createBloomFilterData = spark.udf.register("createBloomFilter", bloomFilterUDF) + bloomFilterDF.withColumn("Data", createBloomFilterData(col("FileName"))) + bloomFilterDF.write.parquet(new Path(indexDataPath, "bf.parquet").toString) } private def write(spark: SparkSession, df: DataFrame, indexConfig: IndexConfig): Unit = { From 98da9413ff956ebb68868be206d691b832cd775d Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Sun, 21 Mar 2021 10:55:53 -0700 Subject: [PATCH 14/16] udf bloom --- .../hyperspace/actions/CreateActionBase.scala | 25 ++++--------------- .../hyperspace/index/CreateIndexTest.scala | 4 ++- 2 files changed, 8 insertions(+), 21 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 31a4f12a3..623e8a664 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -132,8 +132,6 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) } } - - private def write( spark: SparkSession, df: DataFrame, @@ -150,22 +148,6 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) case _ => indexConfig.numBits } - def cleanCountry = (country: String) => { - val allUSA = Seq("US", "USa", "USA", "United states", "United states of America") - if (allUSA.contains(country)) { - "USA" - } - else { - "unknown" - } - } - - val expectedGlobalItems: Long = - df.agg(approx_count_distinct(resolvedIndexedColumn.head)).collect()(0)(0).asInstanceOf[Long] - val globalBF = BloomFilter.create(expectedGlobalItems, resolvedNumBits) - assert( - globalBF.isCompatible(BloomFilter.create(indexConfig.expectedNumItems, resolvedNumBits))) - // TODO Begin has this op as relation is created there // TODO Maybe use lineage to make file smaller val relations = getRelation(spark, df).createRelationMetadata(fileIdTracker) @@ -187,8 +169,11 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) relations.rootPaths.map(p => Tuple1(p)) ).toDF("FileName") val createBloomFilterData = spark.udf.register("createBloomFilter", bloomFilterUDF) - bloomFilterDF.withColumn("Data", createBloomFilterData(col("FileName"))) - bloomFilterDF.write.parquet(new Path(indexDataPath, "bf.parquet").toString) + val bloomFilterResult = bloomFilterDF.withColumn( + "Data", + createBloomFilterData(col("FileName")) + ) + bloomFilterResult.write.parquet(new Path(indexDataPath, "bf.parquet").toString) } private def write(spark: SparkSession, df: DataFrame, indexConfig: IndexConfig): Unit = { diff --git a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala index 95bd2ee03..6d78215e8 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala @@ -39,7 +39,7 @@ class CreateIndexTest extends HyperspaceSuite with SQLHelper { private val indexConfig2 = IndexConfig("index2", Seq("Query"), Seq("imprs")) private val indexConfig3 = IndexConfig("index3", Seq("imprs"), Seq("clicks")) private val indexConfig4 = IndexConfig("index4", Seq("Date", "Query"), Seq("clicks")) - private val bloomIndexConfig = BloomFilterIndexConfig("indexBF1", "Affiliation", 10) + private val bloomIndexConfig = BloomFilterIndexConfig("indexBF1", "Affiliation", 100) private var nonPartitionedDataDF: DataFrame = _ private var partitionedDataDF: DataFrame = _ private var comicDataDF: DataFrame = _ @@ -85,6 +85,8 @@ class CreateIndexTest extends HyperspaceSuite with SQLHelper { test("Creating one bloom filter index.") { hyperspace.createIndex(comicDataDF, bloomIndexConfig) + spark.read.parquet( + systemPath.toString + "\\" + bloomIndexConfig.indexName + "\\v__=0\\bf.parquet").show() val count = hyperspace.indexes.where(s"name = '${bloomIndexConfig.indexName}' ").count assert(count == 1) } From ea42466c21447ddf7fd0d1d46be74f923f745114 Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Mon, 29 Mar 2021 10:16:52 -0700 Subject: [PATCH 15/16] code --- .../hyperspace/actions/CreateActionBase.scala | 4 +- .../index/rules/FilterIndexRule.scala | 22 +++++- .../index/rules/FilterIndexRuleTest.scala | 29 ++++++- .../index/rules/HyperspaceRuleSuite.scala | 76 +++++++++++++++++-- 4 files changed, 116 insertions(+), 15 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 623e8a664..5ed178e5c 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -91,7 +91,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) HyperSpaceIndex.BloomFilterIndex( HyperSpaceIndex.Properties.BloomFilter( HyperSpaceIndex.Properties.CommonProperties - .Columns(resolvedIncludedColumns, resolvedIncludedColumns), + .Columns(resolvedIndexedColumns, resolvedIndexedColumns), IndexLogEntry.schemaString(indexDataFrame.schema), indexProperties)) case _ => throw HyperspaceException("Invalid Index Config.") @@ -305,7 +305,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) val resolvedIndexColumn = Seq(resolveBloomFilterIndexConfig(df, bloomIndexConfig)) val indexDF = resolveDataFrameForLineage(spark, df, resolvedIndexColumn) - (indexDF, resolvedIndexColumn, Seq()) + (indexDF, resolvedIndexColumn, df.columns) } } } diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala index 23b1f5838..7e2cdcc8f 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -21,14 +21,15 @@ import org.apache.spark.sql.catalyst.analysis.CleanupAliases import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule - import com.microsoft.hyperspace.{ActiveSparkSession, Hyperspace} import com.microsoft.hyperspace.actions.Constants -import com.microsoft.hyperspace.index.IndexLogEntry +import com.microsoft.hyperspace.index.HyperSpaceIndex.BloomFilterIndex +import com.microsoft.hyperspace.index.{IndexDataManagerFactoryImpl, IndexLogEntry, PathResolver} import com.microsoft.hyperspace.index.rankers.FilterIndexRanker import com.microsoft.hyperspace.index.sources.FileBasedRelation import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEventLogging, HyperspaceIndexUsageEvent} import com.microsoft.hyperspace.util.{HyperspaceConf, ResolverUtils} +import org.apache.hadoop.fs.Path /** * FilterIndex rule looks for opportunities in a logical plan to replace @@ -59,6 +60,23 @@ object FilterIndexRule // As FilterIndexRule is not intended to support bucketed scan, we set // useBucketUnionForAppended as false. If it's true, Hybrid Scan can cause // unnecessary shuffle for appended data to apply BucketUnion for merging data. + + val files = index.derivedDataset match { + case _: BloomFilterIndex => + // Register udf to use bf, TODO also somehow find index log entry path + val hadoopConf = spark.sessionState.newHadoopConf() + val indexPath = PathResolver(spark.sessionState.conf, hadoopConf) + .getIndexPath(index.name) + val indexManager = IndexDataManagerFactoryImpl.create(indexPath, hadoopConf) + val id = indexManager.getLatestVersionId() + if (id.isDefined) { + val bfPath = new Path(indexManager.getPath(id.get), "bf.parquet") + val bfDF = spark.read.parquet(bfPath.toString) + filter.condition.foldable + } + None + } + val transformedPlan = RuleUtils.transformPlanToUseIndex( spark, diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala index 715f7a177..2c8e59e70 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala @@ -22,8 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, And, AttributeReference import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.types.{IntegerType, StringType, StructType} - +import org.apache.spark.sql.types.{HIVE_TYPE_STRING, IntegerType, StringType, StructType} import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.util.PathUtils @@ -32,15 +31,22 @@ class FilterIndexRuleTest extends HyperspaceRuleSuite { override val systemPath = PathUtils.makeAbsolute("src/test/resources/joinIndexTest") val indexName1 = "filterIxTestIndex1" val indexName2 = "filterIxTestIndex2" + val bloomIndexName = "bloomTestIndex" val c1 = AttributeReference("c1", StringType)() val c2 = AttributeReference("c2", StringType)() val c3 = AttributeReference("c3", StringType)() val c4 = AttributeReference("c4", IntegerType)() + val c5 = AttributeReference("c5", StringType)() val tableSchema = schemaFromAttributes(c1, c2, c3, c4) var scanNode: LogicalRelation = _ + var indexLogEntry1: IndexLogEntry = _ + var indexLogEntry2: IndexLogEntry = _ + var bloomIndexLogEntry: IndexLogEntry = _ + + override def beforeAll(): Unit = { super.beforeAll() @@ -51,10 +57,14 @@ class FilterIndexRuleTest extends HyperspaceRuleSuite { scanNode = LogicalRelation(relation, Seq(c1, c2, c3, c4), None, false) val indexPlan = Project(Seq(c1, c2, c3), scanNode) - createIndexLogEntry(indexName1, Seq(c3, c2), Seq(c1), indexPlan) + indexLogEntry1 = createIndexLogEntry(indexName1, Seq(c3, c2), Seq(c1), indexPlan) val index2Plan = Project(Seq(c1, c2, c3, c4), scanNode) - createIndexLogEntry(indexName2, Seq(c4, c2), Seq(c1, c3), index2Plan) + indexLogEntry2 = createIndexLogEntry(indexName2, Seq(c4, c2), Seq(c1, c3), index2Plan) + + val bloomIndexPlan = Project(Seq(c1, c2, c3, c4), scanNode) + bloomIndexLogEntry = createBloomIndexLogEntry( + bloomIndexName, Seq(c5), Seq(c1, c2, c3, c4, c5), bloomIndexPlan) } before { @@ -72,6 +82,17 @@ class FilterIndexRuleTest extends HyperspaceRuleSuite { verifyTransformedPlanWithIndex(transformedPlan, indexName1) } + test("Verify FilterIndex rule is applied correctly for bloom filter.") { + val filterCondition = And(IsNotNull(c5), EqualTo(c5, Literal("bloom"))) + val filterNode = Filter(filterCondition, scanNode) + + val originalPlan = Project(Seq(c2, c3), filterNode) + val transformedPlan = FilterIndexRule(originalPlan) + + assert(!transformedPlan.equals(originalPlan), "No plan transformation.") + verifyTransformedPlanWithIndex(transformedPlan, bloomIndexName) + } + test("Verify FilterIndex rule is applied correctly for case insensitive query.") { val c2Caps = c2.withName("C2") val c3Caps = c3.withName("C3") diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala index d41480760..9443a8b46 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala @@ -30,14 +30,15 @@ import com.microsoft.hyperspace.index.Hdfs.Properties trait HyperspaceRuleSuite extends HyperspaceSuite { private val filenames = Seq("f1.parquet", "f2.parquet") + def createIndexLogEntry( - name: String, - indexCols: Seq[AttributeReference], - includedCols: Seq[AttributeReference], - plan: LogicalPlan, - numBuckets: Int = 10, - inputFiles: Seq[FileInfo] = Seq(), - writeLog: Boolean = true): IndexLogEntry = { + name: String, + indexCols: Seq[AttributeReference], + includedCols: Seq[AttributeReference], + plan: LogicalPlan, + numBuckets: Int = 10, + inputFiles: Seq[FileInfo] = Seq(), + writeLog: Boolean = true): IndexLogEntry = { val signClass = new RuleTestHelper.TestSignatureProvider().getClass.getName LogicalPlanSignatureProvider.create(signClass).signature(plan) match { @@ -82,6 +83,67 @@ trait HyperspaceRuleSuite extends HyperspaceSuite { } } + def createBloomIndexLogEntry( + name: String, + indexCols: Seq[AttributeReference], + includedCols: Seq[AttributeReference], + plan: LogicalPlan, + inputFiles: Seq[FileInfo] = Seq(), + writeLog: Boolean = true): IndexLogEntry = { + val signClass = new RuleTestHelper.TestSignatureProvider().getClass.getName + + LogicalPlanSignatureProvider.create(signClass).signature(plan) match { + case Some(s) => + val sourcePlanProperties = SparkPlan.Properties( + Seq( + Relation( + Seq("dummy"), + Hdfs(Properties(Content(Directory("/", files = inputFiles)))), + "schema", + "format", + Map())), + null, + null, + LogicalPlanFingerprint(LogicalPlanFingerprint.Properties(Seq(Signature(signClass, s))))) + + val indexFiles = getIndexDataFilesPaths(name).map { path => + new FileStatus(10, false, 1, 10, 10, path) + } + + val indexLogEntry = IndexLogEntry( + name, + HyperSpaceIndex.BloomFilterIndex( + HyperSpaceIndex.Properties.BloomFilter( + HyperSpaceIndex.Properties.CommonProperties + .Columns(indexCols.map(_.name), includedCols.map(_.name)), + IndexLogEntry.schemaString(schemaFromAttributes(indexCols ++ includedCols: _*)), + Map())), + Content.fromLeafFiles(indexFiles, new FileIdTracker).get, + Source(SparkPlan(sourcePlanProperties)), + Map()) + + val logManager = new IndexLogManagerImpl(getIndexRootPath(name)) + indexLogEntry.state = Constants.States.ACTIVE + if (writeLog) { + assert(logManager.writeLog(0, indexLogEntry)) + } + indexLogEntry + + case None => throw HyperspaceException("Invalid plan for index dataFrame.") + } + } + + def getBloomDataFilePath(indexName: String): Option[Path] = { + Some( + new Path ( + new Path ( + new Path (systemPath, indexName), + s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0"), + "bf.parquet" + ) + ) + } + def getIndexDataFilesPaths(indexName: String): Seq[Path] = filenames.map { f => new Path( From c7815ca933022159a392285ce99cd53dfd1010ca Mon Sep 17 00:00:00 2001 From: Gurleen Dhody Date: Wed, 21 Apr 2021 15:01:19 -0700 Subject: [PATCH 16/16] bf run demo test --- .../hyperspace/actions/CreateActionBase.scala | 40 +- .../index/rules/FilterIndexRule.scala | 44 +- src/test/resources/tpcds/queries/q1.sql | 19 - src/test/resources/tpcds/queries/q10.sql | 57 -- src/test/resources/tpcds/queries/q11.sql | 68 -- src/test/resources/tpcds/queries/q12.sql | 22 - src/test/resources/tpcds/queries/q13.sql | 49 - src/test/resources/tpcds/queries/q14a.sql | 120 --- src/test/resources/tpcds/queries/q14b.sql | 95 -- src/test/resources/tpcds/queries/q15.sql | 15 - src/test/resources/tpcds/queries/q16.sql | 23 - src/test/resources/tpcds/queries/q17.sql | 33 - src/test/resources/tpcds/queries/q18.sql | 28 - src/test/resources/tpcds/queries/q19.sql | 19 - src/test/resources/tpcds/queries/q2.sql | 81 -- src/test/resources/tpcds/queries/q20.sql | 18 - src/test/resources/tpcds/queries/q21.sql | 25 - src/test/resources/tpcds/queries/q22.sql | 14 - src/test/resources/tpcds/queries/q23a.sql | 53 - src/test/resources/tpcds/queries/q23b.sql | 68 -- src/test/resources/tpcds/queries/q24a.sql | 34 - src/test/resources/tpcds/queries/q24b.sql | 34 - src/test/resources/tpcds/queries/q25.sql | 33 - src/test/resources/tpcds/queries/q26.sql | 19 - src/test/resources/tpcds/queries/q27.sql | 21 - src/test/resources/tpcds/queries/q28.sql | 56 - src/test/resources/tpcds/queries/q29.sql | 32 - src/test/resources/tpcds/queries/q3.sql | 13 - src/test/resources/tpcds/queries/q30.sql | 35 - src/test/resources/tpcds/queries/q31.sql | 60 -- src/test/resources/tpcds/queries/q32.sql | 15 - src/test/resources/tpcds/queries/q33.sql | 65 -- src/test/resources/tpcds/queries/q34.sql | 32 - src/test/resources/tpcds/queries/q35.sql | 46 - src/test/resources/tpcds/queries/q36.sql | 26 - src/test/resources/tpcds/queries/q37.sql | 15 - src/test/resources/tpcds/queries/q38.sql | 30 - src/test/resources/tpcds/queries/q39a.sql | 47 - src/test/resources/tpcds/queries/q39b.sql | 48 - src/test/resources/tpcds/queries/q4.sql | 120 --- src/test/resources/tpcds/queries/q40.sql | 25 - src/test/resources/tpcds/queries/q41.sql | 49 - src/test/resources/tpcds/queries/q42.sql | 18 - src/test/resources/tpcds/queries/q43.sql | 33 - src/test/resources/tpcds/queries/q44.sql | 46 - src/test/resources/tpcds/queries/q45.sql | 21 - src/test/resources/tpcds/queries/q46.sql | 32 - src/test/resources/tpcds/queries/q47.sql | 63 -- src/test/resources/tpcds/queries/q48.sql | 63 -- src/test/resources/tpcds/queries/q49.sql | 126 --- src/test/resources/tpcds/queries/q5.sql | 131 --- src/test/resources/tpcds/queries/q50.sql | 47 - src/test/resources/tpcds/queries/q51.sql | 55 - src/test/resources/tpcds/queries/q52.sql | 14 - src/test/resources/tpcds/queries/q53.sql | 30 - src/test/resources/tpcds/queries/q54.sql | 61 -- src/test/resources/tpcds/queries/q55.sql | 13 - src/test/resources/tpcds/queries/q56.sql | 65 -- src/test/resources/tpcds/queries/q57.sql | 56 - src/test/resources/tpcds/queries/q58.sql | 59 -- src/test/resources/tpcds/queries/q59.sql | 75 -- src/test/resources/tpcds/queries/q6.sql | 21 - src/test/resources/tpcds/queries/q60.sql | 62 -- src/test/resources/tpcds/queries/q61.sql | 33 - src/test/resources/tpcds/queries/q62.sql | 35 - src/test/resources/tpcds/queries/q63.sql | 31 - src/test/resources/tpcds/queries/q64.sql | 92 -- src/test/resources/tpcds/queries/q65.sql | 33 - src/test/resources/tpcds/queries/q66.sql | 240 ----- src/test/resources/tpcds/queries/q67.sql | 38 - src/test/resources/tpcds/queries/q68.sql | 34 - src/test/resources/tpcds/queries/q69.sql | 38 - src/test/resources/tpcds/queries/q7.sql | 19 - src/test/resources/tpcds/queries/q70.sql | 38 - src/test/resources/tpcds/queries/q71.sql | 44 - src/test/resources/tpcds/queries/q72.sql | 33 - src/test/resources/tpcds/queries/q73.sql | 30 - src/test/resources/tpcds/queries/q74.sql | 58 -- src/test/resources/tpcds/queries/q75.sql | 76 -- src/test/resources/tpcds/queries/q76.sql | 47 - src/test/resources/tpcds/queries/q77.sql | 100 -- src/test/resources/tpcds/queries/q78.sql | 64 -- src/test/resources/tpcds/queries/q79.sql | 27 - src/test/resources/tpcds/queries/q8.sql | 87 -- src/test/resources/tpcds/queries/q80.sql | 94 -- src/test/resources/tpcds/queries/q81.sql | 38 - src/test/resources/tpcds/queries/q82.sql | 15 - src/test/resources/tpcds/queries/q83.sql | 56 - src/test/resources/tpcds/queries/q84.sql | 19 - src/test/resources/tpcds/queries/q85.sql | 82 -- src/test/resources/tpcds/queries/q86.sql | 24 - src/test/resources/tpcds/queries/q87.sql | 28 - src/test/resources/tpcds/queries/q88.sql | 122 --- src/test/resources/tpcds/queries/q89.sql | 30 - src/test/resources/tpcds/queries/q9.sql | 48 - src/test/resources/tpcds/queries/q90.sql | 19 - src/test/resources/tpcds/queries/q91.sql | 23 - src/test/resources/tpcds/queries/q92.sql | 16 - src/test/resources/tpcds/queries/q93.sql | 19 - src/test/resources/tpcds/queries/q94.sql | 23 - src/test/resources/tpcds/queries/q95.sql | 29 - src/test/resources/tpcds/queries/q96.sql | 11 - src/test/resources/tpcds/queries/q97.sql | 30 - src/test/resources/tpcds/queries/q98.sql | 21 - src/test/resources/tpcds/queries/q99.sql | 34 - .../approved-plans-v1_4/q1/explain.txt | 43 - .../approved-plans-v1_4/q1/simplified.txt | 58 -- .../approved-plans-v1_4/q10/explain.txt | 279 ----- .../approved-plans-v1_4/q10/simplified.txt | 74 -- .../approved-plans-v1_4/q11/explain.txt | 415 -------- .../approved-plans-v1_4/q11/simplified.txt | 108 -- .../approved-plans-v1_4/q12/explain.txt | 137 --- .../approved-plans-v1_4/q12/simplified.txt | 38 - .../approved-plans-v1_4/q13/explain.txt | 216 ---- .../approved-plans-v1_4/q13/simplified.txt | 57 -- .../approved-plans-v1_4/q14a/explain.txt | 798 --------------- .../approved-plans-v1_4/q14a/simplified.txt | 214 ---- .../approved-plans-v1_4/q14b/explain.txt | 763 -------------- .../approved-plans-v1_4/q14b/simplified.txt | 204 ---- .../approved-plans-v1_4/q15/explain.txt | 150 --- .../approved-plans-v1_4/q15/simplified.txt | 39 - .../approved-plans-v1_4/q16/explain.txt | 235 ----- .../approved-plans-v1_4/q16/simplified.txt | 62 -- .../approved-plans-v1_4/q17/explain.txt | 269 ----- .../approved-plans-v1_4/q17/simplified.txt | 71 -- .../approved-plans-v1_4/q18/explain.txt | 264 ----- .../approved-plans-v1_4/q18/simplified.txt | 69 -- .../approved-plans-v1_4/q19/explain.txt | 221 ---- .../approved-plans-v1_4/q19/simplified.txt | 58 -- .../approved-plans-v1_4/q2/explain.txt | 218 ---- .../approved-plans-v1_4/q2/simplified.txt | 61 -- .../approved-plans-v1_4/q20/explain.txt | 137 --- .../approved-plans-v1_4/q20/simplified.txt | 38 - .../approved-plans-v1_4/q21/explain.txt | 155 --- .../approved-plans-v1_4/q21/simplified.txt | 40 - .../approved-plans-v1_4/q22/explain.txt | 155 --- .../approved-plans-v1_4/q22/simplified.txt | 40 - .../approved-plans-v1_4/q23a/explain.txt | 537 ---------- .../approved-plans-v1_4/q23a/simplified.txt | 142 --- .../approved-plans-v1_4/q23b/explain.txt | 689 ------------- .../approved-plans-v1_4/q23b/simplified.txt | 182 ---- .../approved-plans-v1_4/q24a/explain.txt | 477 --------- .../approved-plans-v1_4/q24a/simplified.txt | 125 --- .../approved-plans-v1_4/q24b/explain.txt | 477 --------- .../approved-plans-v1_4/q24b/simplified.txt | 125 --- .../approved-plans-v1_4/q25/explain.txt | 269 ----- .../approved-plans-v1_4/q25/simplified.txt | 71 -- .../approved-plans-v1_4/q26/explain.txt | 193 ---- .../approved-plans-v1_4/q26/simplified.txt | 50 - .../approved-plans-v1_4/q27/explain.txt | 193 ---- .../approved-plans-v1_4/q27/simplified.txt | 50 - .../approved-plans-v1_4/q28/explain.txt | 437 -------- .../approved-plans-v1_4/q28/simplified.txt | 107 -- .../approved-plans-v1_4/q29/explain.txt | 292 ------ .../approved-plans-v1_4/q29/simplified.txt | 77 -- .../approved-plans-v1_4/q3/explain.txt | 122 --- .../approved-plans-v1_4/q3/simplified.txt | 31 - .../approved-plans-v1_4/q30/explain.txt | 303 ------ .../approved-plans-v1_4/q30/simplified.txt | 78 -- .../approved-plans-v1_4/q31/explain.txt | 563 ---------- .../approved-plans-v1_4/q31/simplified.txt | 150 --- .../approved-plans-v1_4/q32/explain.txt | 175 ---- .../approved-plans-v1_4/q32/simplified.txt | 45 - .../approved-plans-v1_4/q33/explain.txt | 378 ------- .../approved-plans-v1_4/q33/simplified.txt | 101 -- .../approved-plans-v1_4/q34/explain.txt | 203 ---- .../approved-plans-v1_4/q34/simplified.txt | 54 - .../approved-plans-v1_4/q35/explain.txt | 274 ----- .../approved-plans-v1_4/q35/simplified.txt | 73 -- .../approved-plans-v1_4/q36/explain.txt | 180 ---- .../approved-plans-v1_4/q36/simplified.txt | 49 - .../approved-plans-v1_4/q37/explain.txt | 160 --- .../approved-plans-v1_4/q37/simplified.txt | 41 - .../approved-plans-v1_4/q38/explain.txt | 323 ------ .../approved-plans-v1_4/q38/simplified.txt | 80 -- .../approved-plans-v1_4/q39a/explain.txt | 292 ------ .../approved-plans-v1_4/q39a/simplified.txt | 77 -- .../approved-plans-v1_4/q39b/explain.txt | 292 ------ .../approved-plans-v1_4/q39b/simplified.txt | 77 -- .../approved-plans-v1_4/q4/explain.txt | 606 ----------- .../approved-plans-v1_4/q4/simplified.txt | 158 --- .../approved-plans-v1_4/q40/explain.txt | 183 ---- .../approved-plans-v1_4/q40/simplified.txt | 48 - .../approved-plans-v1_4/q41/explain.txt | 120 --- .../approved-plans-v1_4/q41/simplified.txt | 29 - .../approved-plans-v1_4/q42/explain.txt | 122 --- .../approved-plans-v1_4/q42/simplified.txt | 31 - .../approved-plans-v1_4/q43/explain.txt | 122 --- .../approved-plans-v1_4/q43/simplified.txt | 31 - .../approved-plans-v1_4/q44/explain.txt | 248 ----- .../approved-plans-v1_4/q44/simplified.txt | 68 -- .../approved-plans-v1_4/q45/explain.txt | 226 ----- .../approved-plans-v1_4/q45/simplified.txt | 59 -- .../approved-plans-v1_4/q46/explain.txt | 241 ----- .../approved-plans-v1_4/q46/simplified.txt | 63 -- .../approved-plans-v1_4/q47/explain.txt | 278 ----- .../approved-plans-v1_4/q47/simplified.txt | 84 -- .../approved-plans-v1_4/q48/explain.txt | 183 ---- .../approved-plans-v1_4/q48/simplified.txt | 48 - .../approved-plans-v1_4/q49/explain.txt | 433 -------- .../approved-plans-v1_4/q49/simplified.txt | 126 --- .../approved-plans-v1_4/q5/explain.txt | 435 -------- .../approved-plans-v1_4/q5/simplified.txt | 123 --- .../approved-plans-v1_4/q50/explain.txt | 183 ---- .../approved-plans-v1_4/q50/simplified.txt | 48 - .../approved-plans-v1_4/q51/explain.txt | 228 ----- .../approved-plans-v1_4/q51/simplified.txt | 71 -- .../approved-plans-v1_4/q52/explain.txt | 122 --- .../approved-plans-v1_4/q52/simplified.txt | 31 - .../approved-plans-v1_4/q53/explain.txt | 180 ---- .../approved-plans-v1_4/q53/simplified.txt | 49 - .../approved-plans-v1_4/q54/explain.txt | 459 --------- .../approved-plans-v1_4/q54/simplified.txt | 121 --- .../approved-plans-v1_4/q55/explain.txt | 122 --- .../approved-plans-v1_4/q55/simplified.txt | 31 - .../approved-plans-v1_4/q56/explain.txt | 378 ------- .../approved-plans-v1_4/q56/simplified.txt | 101 -- .../approved-plans-v1_4/q57/explain.txt | 278 ----- .../approved-plans-v1_4/q57/simplified.txt | 84 -- .../approved-plans-v1_4/q58/explain.txt | 477 --------- .../approved-plans-v1_4/q58/simplified.txt | 125 --- .../approved-plans-v1_4/q59/explain.txt | 249 ----- .../approved-plans-v1_4/q59/simplified.txt | 66 -- .../approved-plans-v1_4/q6/explain.txt | 301 ------ .../approved-plans-v1_4/q6/simplified.txt | 77 -- .../approved-plans-v1_4/q60/explain.txt | 378 ------- .../approved-plans-v1_4/q60/simplified.txt | 101 -- .../approved-plans-v1_4/q61/explain.txt | 396 -------- .../approved-plans-v1_4/q61/simplified.txt | 105 -- .../approved-plans-v1_4/q62/explain.txt | 183 ---- .../approved-plans-v1_4/q62/simplified.txt | 48 - .../approved-plans-v1_4/q63/explain.txt | 180 ---- .../approved-plans-v1_4/q63/simplified.txt | 49 - .../approved-plans-v1_4/q64/explain.txt | 918 ----------------- .../approved-plans-v1_4/q64/simplified.txt | 246 ----- .../approved-plans-v1_4/q65/explain.txt | 245 ----- .../approved-plans-v1_4/q65/simplified.txt | 63 -- .../approved-plans-v1_4/q66/explain.txt | 310 ------ .../approved-plans-v1_4/q66/simplified.txt | 83 -- .../approved-plans-v1_4/q67/explain.txt | 175 ---- .../approved-plans-v1_4/q67/simplified.txt | 48 - .../approved-plans-v1_4/q68/explain.txt | 241 ----- .../approved-plans-v1_4/q68/simplified.txt | 63 -- .../approved-plans-v1_4/q69/explain.txt | 274 ----- .../approved-plans-v1_4/q69/simplified.txt | 73 -- .../approved-plans-v1_4/q7/explain.txt | 193 ---- .../approved-plans-v1_4/q7/simplified.txt | 50 - .../approved-plans-v1_4/q70/explain.txt | 264 ----- .../approved-plans-v1_4/q70/simplified.txt | 74 -- .../approved-plans-v1_4/q71/explain.txt | 232 ----- .../approved-plans-v1_4/q71/simplified.txt | 65 -- .../approved-plans-v1_4/q72/explain.txt | 391 ------- .../approved-plans-v1_4/q72/simplified.txt | 104 -- .../approved-plans-v1_4/q73/explain.txt | 203 ---- .../approved-plans-v1_4/q73/simplified.txt | 54 - .../approved-plans-v1_4/q74/explain.txt | 410 -------- .../approved-plans-v1_4/q74/simplified.txt | 107 -- .../approved-plans-v1_4/q75/explain.txt | 647 ------------ .../approved-plans-v1_4/q75/simplified.txt | 180 ---- .../approved-plans-v1_4/q76/explain.txt | 209 ---- .../approved-plans-v1_4/q76/simplified.txt | 58 -- .../approved-plans-v1_4/q77/explain.txt | 520 ---------- .../approved-plans-v1_4/q77/simplified.txt | 139 --- .../approved-plans-v1_4/q78/explain.txt | 341 ------- .../approved-plans-v1_4/q78/simplified.txt | 88 -- .../approved-plans-v1_4/q79/explain.txt | 193 ---- .../approved-plans-v1_4/q79/simplified.txt | 50 - .../approved-plans-v1_4/q8/explain.txt | 272 ----- .../approved-plans-v1_4/q8/simplified.txt | 70 -- .../approved-plans-v1_4/q80/explain.txt | 553 ---------- .../approved-plans-v1_4/q80/simplified.txt | 148 --- .../approved-plans-v1_4/q81/explain.txt | 298 ------ .../approved-plans-v1_4/q81/simplified.txt | 77 -- .../approved-plans-v1_4/q82/explain.txt | 160 --- .../approved-plans-v1_4/q82/simplified.txt | 41 - .../approved-plans-v1_4/q83/explain.txt | 344 ------- .../approved-plans-v1_4/q83/simplified.txt | 91 -- .../approved-plans-v1_4/q84/explain.txt | 200 ---- .../approved-plans-v1_4/q84/simplified.txt | 53 - .../approved-plans-v1_4/q85/explain.txt | 287 ------ .../approved-plans-v1_4/q85/simplified.txt | 76 -- .../approved-plans-v1_4/q86/explain.txt | 142 --- .../approved-plans-v1_4/q86/simplified.txt | 39 - .../approved-plans-v1_4/q87/explain.txt | 323 ------ .../approved-plans-v1_4/q87/simplified.txt | 80 -- .../approved-plans-v1_4/q88/explain.txt | 960 ------------------ .../approved-plans-v1_4/q88/simplified.txt | 250 ----- .../approved-plans-v1_4/q89/explain.txt | 175 ---- .../approved-plans-v1_4/q89/simplified.txt | 48 - .../approved-plans-v1_4/q9/explain.txt | 718 ------------- .../approved-plans-v1_4/q9/simplified.txt | 186 ---- .../approved-plans-v1_4/q90/explain.txt | 280 ----- .../approved-plans-v1_4/q90/simplified.txt | 74 -- .../approved-plans-v1_4/q91/explain.txt | 264 ----- .../approved-plans-v1_4/q91/simplified.txt | 71 -- .../approved-plans-v1_4/q92/explain.txt | 196 ---- .../approved-plans-v1_4/q92/simplified.txt | 50 - .../approved-plans-v1_4/q93/explain.txt | 111 -- .../approved-plans-v1_4/q93/simplified.txt | 29 - .../approved-plans-v1_4/q94/explain.txt | 235 ----- .../approved-plans-v1_4/q94/simplified.txt | 62 -- .../approved-plans-v1_4/q95/explain.txt | 318 ------ .../approved-plans-v1_4/q95/simplified.txt | 84 -- .../approved-plans-v1_4/q96/explain.txt | 160 --- .../approved-plans-v1_4/q96/simplified.txt | 41 - .../approved-plans-v1_4/q97/explain.txt | 174 ---- .../approved-plans-v1_4/q97/simplified.txt | 45 - .../approved-plans-v1_4/q98/explain.txt | 147 --- .../approved-plans-v1_4/q98/simplified.txt | 42 - .../approved-plans-v1_4/q99/explain.txt | 183 ---- .../approved-plans-v1_4/q99/simplified.txt | 48 - .../hyperspace/index/CreateIndexTest.scala | 60 +- .../index/rules/FilterIndexRuleTest.scala | 1 + .../index/rules/HyperspaceRuleSuite.scala | 6 +- 314 files changed, 112 insertions(+), 43872 deletions(-) delete mode 100644 src/test/resources/tpcds/queries/q1.sql delete mode 100644 src/test/resources/tpcds/queries/q10.sql delete mode 100644 src/test/resources/tpcds/queries/q11.sql delete mode 100644 src/test/resources/tpcds/queries/q12.sql delete mode 100644 src/test/resources/tpcds/queries/q13.sql delete mode 100644 src/test/resources/tpcds/queries/q14a.sql delete mode 100644 src/test/resources/tpcds/queries/q14b.sql delete mode 100644 src/test/resources/tpcds/queries/q15.sql delete mode 100644 src/test/resources/tpcds/queries/q16.sql delete mode 100644 src/test/resources/tpcds/queries/q17.sql delete mode 100644 src/test/resources/tpcds/queries/q18.sql delete mode 100644 src/test/resources/tpcds/queries/q19.sql delete mode 100644 src/test/resources/tpcds/queries/q2.sql delete mode 100644 src/test/resources/tpcds/queries/q20.sql delete mode 100644 src/test/resources/tpcds/queries/q21.sql delete mode 100644 src/test/resources/tpcds/queries/q22.sql delete mode 100644 src/test/resources/tpcds/queries/q23a.sql delete mode 100644 src/test/resources/tpcds/queries/q23b.sql delete mode 100644 src/test/resources/tpcds/queries/q24a.sql delete mode 100644 src/test/resources/tpcds/queries/q24b.sql delete mode 100644 src/test/resources/tpcds/queries/q25.sql delete mode 100644 src/test/resources/tpcds/queries/q26.sql delete mode 100644 src/test/resources/tpcds/queries/q27.sql delete mode 100644 src/test/resources/tpcds/queries/q28.sql delete mode 100644 src/test/resources/tpcds/queries/q29.sql delete mode 100644 src/test/resources/tpcds/queries/q3.sql delete mode 100644 src/test/resources/tpcds/queries/q30.sql delete mode 100644 src/test/resources/tpcds/queries/q31.sql delete mode 100644 src/test/resources/tpcds/queries/q32.sql delete mode 100644 src/test/resources/tpcds/queries/q33.sql delete mode 100644 src/test/resources/tpcds/queries/q34.sql delete mode 100644 src/test/resources/tpcds/queries/q35.sql delete mode 100644 src/test/resources/tpcds/queries/q36.sql delete mode 100644 src/test/resources/tpcds/queries/q37.sql delete mode 100644 src/test/resources/tpcds/queries/q38.sql delete mode 100644 src/test/resources/tpcds/queries/q39a.sql delete mode 100644 src/test/resources/tpcds/queries/q39b.sql delete mode 100644 src/test/resources/tpcds/queries/q4.sql delete mode 100644 src/test/resources/tpcds/queries/q40.sql delete mode 100644 src/test/resources/tpcds/queries/q41.sql delete mode 100644 src/test/resources/tpcds/queries/q42.sql delete mode 100644 src/test/resources/tpcds/queries/q43.sql delete mode 100644 src/test/resources/tpcds/queries/q44.sql delete mode 100644 src/test/resources/tpcds/queries/q45.sql delete mode 100644 src/test/resources/tpcds/queries/q46.sql delete mode 100644 src/test/resources/tpcds/queries/q47.sql delete mode 100644 src/test/resources/tpcds/queries/q48.sql delete mode 100644 src/test/resources/tpcds/queries/q49.sql delete mode 100644 src/test/resources/tpcds/queries/q5.sql delete mode 100644 src/test/resources/tpcds/queries/q50.sql delete mode 100644 src/test/resources/tpcds/queries/q51.sql delete mode 100644 src/test/resources/tpcds/queries/q52.sql delete mode 100644 src/test/resources/tpcds/queries/q53.sql delete mode 100644 src/test/resources/tpcds/queries/q54.sql delete mode 100644 src/test/resources/tpcds/queries/q55.sql delete mode 100644 src/test/resources/tpcds/queries/q56.sql delete mode 100644 src/test/resources/tpcds/queries/q57.sql delete mode 100644 src/test/resources/tpcds/queries/q58.sql delete mode 100644 src/test/resources/tpcds/queries/q59.sql delete mode 100644 src/test/resources/tpcds/queries/q6.sql delete mode 100644 src/test/resources/tpcds/queries/q60.sql delete mode 100644 src/test/resources/tpcds/queries/q61.sql delete mode 100644 src/test/resources/tpcds/queries/q62.sql delete mode 100644 src/test/resources/tpcds/queries/q63.sql delete mode 100644 src/test/resources/tpcds/queries/q64.sql delete mode 100644 src/test/resources/tpcds/queries/q65.sql delete mode 100644 src/test/resources/tpcds/queries/q66.sql delete mode 100644 src/test/resources/tpcds/queries/q67.sql delete mode 100644 src/test/resources/tpcds/queries/q68.sql delete mode 100644 src/test/resources/tpcds/queries/q69.sql delete mode 100644 src/test/resources/tpcds/queries/q7.sql delete mode 100644 src/test/resources/tpcds/queries/q70.sql delete mode 100644 src/test/resources/tpcds/queries/q71.sql delete mode 100644 src/test/resources/tpcds/queries/q72.sql delete mode 100644 src/test/resources/tpcds/queries/q73.sql delete mode 100644 src/test/resources/tpcds/queries/q74.sql delete mode 100644 src/test/resources/tpcds/queries/q75.sql delete mode 100644 src/test/resources/tpcds/queries/q76.sql delete mode 100644 src/test/resources/tpcds/queries/q77.sql delete mode 100644 src/test/resources/tpcds/queries/q78.sql delete mode 100644 src/test/resources/tpcds/queries/q79.sql delete mode 100644 src/test/resources/tpcds/queries/q8.sql delete mode 100644 src/test/resources/tpcds/queries/q80.sql delete mode 100644 src/test/resources/tpcds/queries/q81.sql delete mode 100644 src/test/resources/tpcds/queries/q82.sql delete mode 100644 src/test/resources/tpcds/queries/q83.sql delete mode 100644 src/test/resources/tpcds/queries/q84.sql delete mode 100644 src/test/resources/tpcds/queries/q85.sql delete mode 100644 src/test/resources/tpcds/queries/q86.sql delete mode 100644 src/test/resources/tpcds/queries/q87.sql delete mode 100644 src/test/resources/tpcds/queries/q88.sql delete mode 100644 src/test/resources/tpcds/queries/q89.sql delete mode 100644 src/test/resources/tpcds/queries/q9.sql delete mode 100644 src/test/resources/tpcds/queries/q90.sql delete mode 100644 src/test/resources/tpcds/queries/q91.sql delete mode 100644 src/test/resources/tpcds/queries/q92.sql delete mode 100644 src/test/resources/tpcds/queries/q93.sql delete mode 100644 src/test/resources/tpcds/queries/q94.sql delete mode 100644 src/test/resources/tpcds/queries/q95.sql delete mode 100644 src/test/resources/tpcds/queries/q96.sql delete mode 100644 src/test/resources/tpcds/queries/q97.sql delete mode 100644 src/test/resources/tpcds/queries/q98.sql delete mode 100644 src/test/resources/tpcds/queries/q99.sql delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q1/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q1/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q10/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q10/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q11/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q11/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q12/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q12/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q13/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q13/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14a/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14a/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14b/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14b/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q15/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q15/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q16/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q16/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q17/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q17/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q18/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q18/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q19/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q19/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q2/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q2/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q20/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q20/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q21/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q21/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q22/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q22/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23a/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23a/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23b/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23b/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24a/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24a/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24b/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24b/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q25/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q25/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q26/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q26/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q27/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q27/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q28/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q28/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q29/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q29/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q3/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q3/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q30/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q30/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q31/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q31/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q32/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q32/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q33/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q33/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q34/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q34/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q35/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q35/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q36/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q36/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q37/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q37/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q38/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q38/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39a/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39a/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39b/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39b/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q4/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q4/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q40/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q40/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q41/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q41/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q42/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q42/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q43/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q43/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q44/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q44/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q45/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q45/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q46/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q46/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q47/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q47/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q48/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q48/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q49/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q49/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q5/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q5/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q50/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q50/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q51/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q51/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q52/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q52/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q53/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q53/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q54/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q54/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q55/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q55/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q56/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q56/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q57/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q57/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q58/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q58/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q59/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q59/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q6/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q6/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q60/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q60/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q61/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q61/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q62/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q62/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q63/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q63/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q64/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q64/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q65/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q65/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q66/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q66/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q67/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q67/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q68/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q68/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q69/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q69/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q7/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q7/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q70/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q70/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q71/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q71/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q72/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q72/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q73/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q73/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q74/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q74/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q75/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q75/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q76/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q76/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q77/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q77/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q78/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q78/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q79/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q79/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q8/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q8/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q80/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q80/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q81/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q81/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q82/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q82/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q83/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q83/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q84/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q84/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q85/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q85/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q86/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q86/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q87/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q87/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q88/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q88/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q89/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q89/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q9/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q9/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q90/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q90/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q91/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q91/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q92/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q92/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q93/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q93/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q94/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q94/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q95/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q95/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q96/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q96/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q97/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q97/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q98/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q98/simplified.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q99/explain.txt delete mode 100644 src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q99/simplified.txt diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 6acc54354..0030869f8 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -151,29 +151,25 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) // TODO Begin has this op as relation is created there // TODO Maybe use lineage to make file smaller val relations = getRelation(spark, df).createRelationMetadata(fileIdTracker) - val bloomFilterUDF = udf((path: String) => { - val bfByteStream = new ByteArrayOutputStream() - val localBF = spark.read.schema(df.schema) - .format(relations.fileFormat) - .options(relations.options) - .load(path) - .select(resolvedIndexedColumn.head) - .stat - .bloomFilter(resolvedIndexedColumn.head, indexConfig.expectedNumItems, resolvedNumBits) - localBF.writeTo(bfByteStream) - bfByteStream.close() - bfByteStream.toByteArray.map(_.toChar).mkString - }) - - val bloomFilterDF = spark.createDataFrame( - relations.rootPaths.map(p => Tuple1(p)) - ).toDF("FileName") - val createBloomFilterData = spark.udf.register("createBloomFilter", bloomFilterUDF) - val bloomFilterResult = bloomFilterDF.withColumn( - "Data", - createBloomFilterData(col("FileName")) + val bloomData = relations.rootPaths.par.map( + path => { + val bfByteStream = new ByteArrayOutputStream() + val localBF = spark.read.schema(df.schema) + .format(relations.fileFormat) + .options(relations.options) + .load(path) + .select(resolvedIndexedColumn.head) + .stat + .bloomFilter(resolvedIndexedColumn.head, indexConfig.expectedNumItems, resolvedNumBits) + localBF.writeTo(bfByteStream) + bfByteStream.close() + (path, bfByteStream.toByteArray.map(_.toChar).mkString) + } ) - bloomFilterResult.write.parquet(new Path(indexDataPath, "bf.parquet").toString) + val bloomDF = spark.createDataFrame( + bloomData.seq + ).toDF("FileName", "Data") + bloomDF.write.parquet(new Path(indexDataPath, "bf.parquet").toString) } protected def write(spark: SparkSession, df: DataFrame, indexConfig: IndexConfig): Unit = { diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala index 7e2cdcc8f..ab2f53b63 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -18,7 +18,7 @@ package com.microsoft.hyperspace.index.rules import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.CleanupAliases -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualNullSafe, EqualTo, Expression} import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import com.microsoft.hyperspace.{ActiveSparkSession, Hyperspace} @@ -26,10 +26,16 @@ import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index.HyperSpaceIndex.BloomFilterIndex import com.microsoft.hyperspace.index.{IndexDataManagerFactoryImpl, IndexLogEntry, PathResolver} import com.microsoft.hyperspace.index.rankers.FilterIndexRanker +import com.microsoft.hyperspace.index.rules.JoinIndexRule.extractConditions import com.microsoft.hyperspace.index.sources.FileBasedRelation import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEventLogging, HyperspaceIndexUsageEvent} import com.microsoft.hyperspace.util.{HyperspaceConf, ResolverUtils} +import com.sun.xml.internal.messaging.saaj.util.ByteInputStream import org.apache.hadoop.fs.Path +import org.apache.spark.sql.functions.udf +import org.apache.spark.util.sketch.BloomFilter + +import java.io.ByteArrayInputStream /** * FilterIndex rule looks for opportunities in a logical plan to replace @@ -62,19 +68,20 @@ object FilterIndexRule // unnecessary shuffle for appended data to apply BucketUnion for merging data. val files = index.derivedDataset match { - case _: BloomFilterIndex => - // Register udf to use bf, TODO also somehow find index log entry path - val hadoopConf = spark.sessionState.newHadoopConf() - val indexPath = PathResolver(spark.sessionState.conf, hadoopConf) - .getIndexPath(index.name) - val indexManager = IndexDataManagerFactoryImpl.create(indexPath, hadoopConf) - val id = indexManager.getLatestVersionId() - if (id.isDefined) { - val bfPath = new Path(indexManager.getPath(id.get), "bf.parquet") - val bfDF = spark.read.parquet(bfPath.toString) - filter.condition.foldable + case bfi: BloomFilterIndex => + // Register udf to use bf, TODO also somehow find index log entry path. + /* + val bfPath = new Path(index.content.files.filter(_.toString.contains("bf.parquet")).head) + val bfDF = spark.read.parquet(bfPath.toString) + val expression = extractConditions(filter.condition) + if (expression.isDefined) { + udf((bfData: String) => { + val bf = BloomFilter.readFrom(new ByteArrayInputStream(bfData.getBytes())) + bf.mightContain("") + }) } None + */ } val transformedPlan = @@ -105,6 +112,19 @@ object FilterIndexRule } } + private def extractConditions(condition: Expression): Option[Expression] = condition match { + case EqualTo(_: AttributeReference, _: AttributeReference) => Some(condition) + case EqualNullSafe(_: AttributeReference, _: AttributeReference) => None + case And(left, right) => + val leaf = extractConditions(left) + if (leaf.isDefined) { + leaf + } else { + extractConditions(right) + } + case _ => throw new IllegalStateException("Unsupported condition found") + } + /** * For a given relation, find all available indexes on it which fully cover given output and * filter columns. diff --git a/src/test/resources/tpcds/queries/q1.sql b/src/test/resources/tpcds/queries/q1.sql deleted file mode 100644 index 4d20faad8..000000000 --- a/src/test/resources/tpcds/queries/q1.sql +++ /dev/null @@ -1,19 +0,0 @@ -WITH customer_total_return AS -( SELECT - sr_customer_sk AS ctr_customer_sk, - sr_store_sk AS ctr_store_sk, - sum(sr_return_amt) AS ctr_total_return - FROM store_returns, date_dim - WHERE sr_returned_date_sk = d_date_sk AND d_year = 2000 - GROUP BY sr_customer_sk, sr_store_sk) -SELECT c_customer_id -FROM customer_total_return ctr1, store, customer -WHERE ctr1.ctr_total_return > - (SELECT avg(ctr_total_return) * 1.2 - FROM customer_total_return ctr2 - WHERE ctr1.ctr_store_sk = ctr2.ctr_store_sk) - AND s_store_sk = ctr1.ctr_store_sk - AND s_state = 'TN' - AND ctr1.ctr_customer_sk = c_customer_sk -ORDER BY c_customer_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q10.sql b/src/test/resources/tpcds/queries/q10.sql deleted file mode 100644 index 5500e1aea..000000000 --- a/src/test/resources/tpcds/queries/q10.sql +++ /dev/null @@ -1,57 +0,0 @@ -SELECT - cd_gender, - cd_marital_status, - cd_education_status, - count(*) cnt1, - cd_purchase_estimate, - count(*) cnt2, - cd_credit_rating, - count(*) cnt3, - cd_dep_count, - count(*) cnt4, - cd_dep_employed_count, - count(*) cnt5, - cd_dep_college_count, - count(*) cnt6 -FROM - customer c, customer_address ca, customer_demographics -WHERE - c.c_current_addr_sk = ca.ca_address_sk AND - ca_county IN ('Rush County', 'Toole County', 'Jefferson County', - 'Dona Ana County', 'La Porte County') AND - cd_demo_sk = c.c_current_cdemo_sk AND - exists(SELECT * - FROM store_sales, date_dim - WHERE c.c_customer_sk = ss_customer_sk AND - ss_sold_date_sk = d_date_sk AND - d_year = 2002 AND - d_moy BETWEEN 1 AND 1 + 3) AND - (exists(SELECT * - FROM web_sales, date_dim - WHERE c.c_customer_sk = ws_bill_customer_sk AND - ws_sold_date_sk = d_date_sk AND - d_year = 2002 AND - d_moy BETWEEN 1 AND 1 + 3) OR - exists(SELECT * - FROM catalog_sales, date_dim - WHERE c.c_customer_sk = cs_ship_customer_sk AND - cs_sold_date_sk = d_date_sk AND - d_year = 2002 AND - d_moy BETWEEN 1 AND 1 + 3)) -GROUP BY cd_gender, - cd_marital_status, - cd_education_status, - cd_purchase_estimate, - cd_credit_rating, - cd_dep_count, - cd_dep_employed_count, - cd_dep_college_count -ORDER BY cd_gender, - cd_marital_status, - cd_education_status, - cd_purchase_estimate, - cd_credit_rating, - cd_dep_count, - cd_dep_employed_count, - cd_dep_college_count -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q11.sql b/src/test/resources/tpcds/queries/q11.sql deleted file mode 100644 index 3618fb14f..000000000 --- a/src/test/resources/tpcds/queries/q11.sql +++ /dev/null @@ -1,68 +0,0 @@ -WITH year_total AS ( - SELECT - c_customer_id customer_id, - c_first_name customer_first_name, - c_last_name customer_last_name, - c_preferred_cust_flag customer_preferred_cust_flag, - c_birth_country customer_birth_country, - c_login customer_login, - c_email_address customer_email_address, - d_year dyear, - sum(ss_ext_list_price - ss_ext_discount_amt) year_total, - 's' sale_type - FROM customer, store_sales, date_dim - WHERE c_customer_sk = ss_customer_sk - AND ss_sold_date_sk = d_date_sk - GROUP BY c_customer_id - , c_first_name - , c_last_name - , d_year - , c_preferred_cust_flag - , c_birth_country - , c_login - , c_email_address - , d_year - UNION ALL - SELECT - c_customer_id customer_id, - c_first_name customer_first_name, - c_last_name customer_last_name, - c_preferred_cust_flag customer_preferred_cust_flag, - c_birth_country customer_birth_country, - c_login customer_login, - c_email_address customer_email_address, - d_year dyear, - sum(ws_ext_list_price - ws_ext_discount_amt) year_total, - 'w' sale_type - FROM customer, web_sales, date_dim - WHERE c_customer_sk = ws_bill_customer_sk - AND ws_sold_date_sk = d_date_sk - GROUP BY - c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, - c_login, c_email_address, d_year) -SELECT t_s_secyear.customer_preferred_cust_flag -FROM year_total t_s_firstyear - , year_total t_s_secyear - , year_total t_w_firstyear - , year_total t_w_secyear -WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id - AND t_s_firstyear.customer_id = t_w_secyear.customer_id - AND t_s_firstyear.customer_id = t_w_firstyear.customer_id - AND t_s_firstyear.sale_type = 's' - AND t_w_firstyear.sale_type = 'w' - AND t_s_secyear.sale_type = 's' - AND t_w_secyear.sale_type = 'w' - AND t_s_firstyear.dyear = 2001 - AND t_s_secyear.dyear = 2001 + 1 - AND t_w_firstyear.dyear = 2001 - AND t_w_secyear.dyear = 2001 + 1 - AND t_s_firstyear.year_total > 0 - AND t_w_firstyear.year_total > 0 - AND CASE WHEN t_w_firstyear.year_total > 0 - THEN t_w_secyear.year_total / t_w_firstyear.year_total - ELSE NULL END - > CASE WHEN t_s_firstyear.year_total > 0 - THEN t_s_secyear.year_total / t_s_firstyear.year_total - ELSE NULL END -ORDER BY t_s_secyear.customer_preferred_cust_flag -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q12.sql b/src/test/resources/tpcds/queries/q12.sql deleted file mode 100644 index 0382737f5..000000000 --- a/src/test/resources/tpcds/queries/q12.sql +++ /dev/null @@ -1,22 +0,0 @@ -SELECT - i_item_desc, - i_category, - i_class, - i_current_price, - sum(ws_ext_sales_price) AS itemrevenue, - sum(ws_ext_sales_price) * 100 / sum(sum(ws_ext_sales_price)) - OVER - (PARTITION BY i_class) AS revenueratio -FROM - web_sales, item, date_dim -WHERE - ws_item_sk = i_item_sk - AND i_category IN ('Sports', 'Books', 'Home') - AND ws_sold_date_sk = d_date_sk - AND d_date BETWEEN cast('1999-02-22' AS DATE) - AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) -GROUP BY - i_item_id, i_item_desc, i_category, i_class, i_current_price -ORDER BY - i_category, i_class, i_item_id, i_item_desc, revenueratio -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q13.sql b/src/test/resources/tpcds/queries/q13.sql deleted file mode 100644 index 32dc9e260..000000000 --- a/src/test/resources/tpcds/queries/q13.sql +++ /dev/null @@ -1,49 +0,0 @@ -SELECT - avg(ss_quantity), - avg(ss_ext_sales_price), - avg(ss_ext_wholesale_cost), - sum(ss_ext_wholesale_cost) -FROM store_sales - , store - , customer_demographics - , household_demographics - , customer_address - , date_dim -WHERE s_store_sk = ss_store_sk - AND ss_sold_date_sk = d_date_sk AND d_year = 2001 - AND ((ss_hdemo_sk = hd_demo_sk - AND cd_demo_sk = ss_cdemo_sk - AND cd_marital_status = 'M' - AND cd_education_status = 'Advanced Degree' - AND ss_sales_price BETWEEN 100.00 AND 150.00 - AND hd_dep_count = 3 -) OR - (ss_hdemo_sk = hd_demo_sk - AND cd_demo_sk = ss_cdemo_sk - AND cd_marital_status = 'S' - AND cd_education_status = 'College' - AND ss_sales_price BETWEEN 50.00 AND 100.00 - AND hd_dep_count = 1 - ) OR - (ss_hdemo_sk = hd_demo_sk - AND cd_demo_sk = ss_cdemo_sk - AND cd_marital_status = 'W' - AND cd_education_status = '2 yr Degree' - AND ss_sales_price BETWEEN 150.00 AND 200.00 - AND hd_dep_count = 1 - )) - AND ((ss_addr_sk = ca_address_sk - AND ca_country = 'United States' - AND ca_state IN ('TX', 'OH', 'TX') - AND ss_net_profit BETWEEN 100 AND 200 -) OR - (ss_addr_sk = ca_address_sk - AND ca_country = 'United States' - AND ca_state IN ('OR', 'NM', 'KY') - AND ss_net_profit BETWEEN 150 AND 300 - ) OR - (ss_addr_sk = ca_address_sk - AND ca_country = 'United States' - AND ca_state IN ('VA', 'TX', 'MS') - AND ss_net_profit BETWEEN 50 AND 250 - )) diff --git a/src/test/resources/tpcds/queries/q14a.sql b/src/test/resources/tpcds/queries/q14a.sql deleted file mode 100644 index 954ddd41b..000000000 --- a/src/test/resources/tpcds/queries/q14a.sql +++ /dev/null @@ -1,120 +0,0 @@ -WITH cross_items AS -(SELECT i_item_sk ss_item_sk - FROM item, - (SELECT - iss.i_brand_id brand_id, - iss.i_class_id class_id, - iss.i_category_id category_id - FROM store_sales, item iss, date_dim d1 - WHERE ss_item_sk = iss.i_item_sk - AND ss_sold_date_sk = d1.d_date_sk - AND d1.d_year BETWEEN 1999 AND 1999 + 2 - INTERSECT - SELECT - ics.i_brand_id, - ics.i_class_id, - ics.i_category_id - FROM catalog_sales, item ics, date_dim d2 - WHERE cs_item_sk = ics.i_item_sk - AND cs_sold_date_sk = d2.d_date_sk - AND d2.d_year BETWEEN 1999 AND 1999 + 2 - INTERSECT - SELECT - iws.i_brand_id, - iws.i_class_id, - iws.i_category_id - FROM web_sales, item iws, date_dim d3 - WHERE ws_item_sk = iws.i_item_sk - AND ws_sold_date_sk = d3.d_date_sk - AND d3.d_year BETWEEN 1999 AND 1999 + 2) x - WHERE i_brand_id = brand_id - AND i_class_id = class_id - AND i_category_id = category_id -), - avg_sales AS - (SELECT avg(quantity * list_price) average_sales - FROM ( - SELECT - ss_quantity quantity, - ss_list_price list_price - FROM store_sales, date_dim - WHERE ss_sold_date_sk = d_date_sk - AND d_year BETWEEN 1999 AND 2001 - UNION ALL - SELECT - cs_quantity quantity, - cs_list_price list_price - FROM catalog_sales, date_dim - WHERE cs_sold_date_sk = d_date_sk - AND d_year BETWEEN 1999 AND 1999 + 2 - UNION ALL - SELECT - ws_quantity quantity, - ws_list_price list_price - FROM web_sales, date_dim - WHERE ws_sold_date_sk = d_date_sk - AND d_year BETWEEN 1999 AND 1999 + 2) x) -SELECT - channel, - i_brand_id, - i_class_id, - i_category_id, - sum(sales), - sum(number_sales) -FROM ( - SELECT - 'store' channel, - i_brand_id, - i_class_id, - i_category_id, - sum(ss_quantity * ss_list_price) sales, - count(*) number_sales - FROM store_sales, item, date_dim - WHERE ss_item_sk IN (SELECT ss_item_sk - FROM cross_items) - AND ss_item_sk = i_item_sk - AND ss_sold_date_sk = d_date_sk - AND d_year = 1999 + 2 - AND d_moy = 11 - GROUP BY i_brand_id, i_class_id, i_category_id - HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales - FROM avg_sales) - UNION ALL - SELECT - 'catalog' channel, - i_brand_id, - i_class_id, - i_category_id, - sum(cs_quantity * cs_list_price) sales, - count(*) number_sales - FROM catalog_sales, item, date_dim - WHERE cs_item_sk IN (SELECT ss_item_sk - FROM cross_items) - AND cs_item_sk = i_item_sk - AND cs_sold_date_sk = d_date_sk - AND d_year = 1999 + 2 - AND d_moy = 11 - GROUP BY i_brand_id, i_class_id, i_category_id - HAVING sum(cs_quantity * cs_list_price) > (SELECT average_sales FROM avg_sales) - UNION ALL - SELECT - 'web' channel, - i_brand_id, - i_class_id, - i_category_id, - sum(ws_quantity * ws_list_price) sales, - count(*) number_sales - FROM web_sales, item, date_dim - WHERE ws_item_sk IN (SELECT ss_item_sk - FROM cross_items) - AND ws_item_sk = i_item_sk - AND ws_sold_date_sk = d_date_sk - AND d_year = 1999 + 2 - AND d_moy = 11 - GROUP BY i_brand_id, i_class_id, i_category_id - HAVING sum(ws_quantity * ws_list_price) > (SELECT average_sales - FROM avg_sales) - ) y -GROUP BY ROLLUP (channel, i_brand_id, i_class_id, i_category_id) -ORDER BY channel, i_brand_id, i_class_id, i_category_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q14b.sql b/src/test/resources/tpcds/queries/q14b.sql deleted file mode 100644 index 929a8484b..000000000 --- a/src/test/resources/tpcds/queries/q14b.sql +++ /dev/null @@ -1,95 +0,0 @@ -WITH cross_items AS -(SELECT i_item_sk ss_item_sk - FROM item, - (SELECT - iss.i_brand_id brand_id, - iss.i_class_id class_id, - iss.i_category_id category_id - FROM store_sales, item iss, date_dim d1 - WHERE ss_item_sk = iss.i_item_sk - AND ss_sold_date_sk = d1.d_date_sk - AND d1.d_year BETWEEN 1999 AND 1999 + 2 - INTERSECT - SELECT - ics.i_brand_id, - ics.i_class_id, - ics.i_category_id - FROM catalog_sales, item ics, date_dim d2 - WHERE cs_item_sk = ics.i_item_sk - AND cs_sold_date_sk = d2.d_date_sk - AND d2.d_year BETWEEN 1999 AND 1999 + 2 - INTERSECT - SELECT - iws.i_brand_id, - iws.i_class_id, - iws.i_category_id - FROM web_sales, item iws, date_dim d3 - WHERE ws_item_sk = iws.i_item_sk - AND ws_sold_date_sk = d3.d_date_sk - AND d3.d_year BETWEEN 1999 AND 1999 + 2) x - WHERE i_brand_id = brand_id - AND i_class_id = class_id - AND i_category_id = category_id -), - avg_sales AS - (SELECT avg(quantity * list_price) average_sales - FROM (SELECT - ss_quantity quantity, - ss_list_price list_price - FROM store_sales, date_dim - WHERE ss_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2 - UNION ALL - SELECT - cs_quantity quantity, - cs_list_price list_price - FROM catalog_sales, date_dim - WHERE cs_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2 - UNION ALL - SELECT - ws_quantity quantity, - ws_list_price list_price - FROM web_sales, date_dim - WHERE ws_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2) x) -SELECT * -FROM - (SELECT - 'store' channel, - i_brand_id, - i_class_id, - i_category_id, - sum(ss_quantity * ss_list_price) sales, - count(*) number_sales - FROM store_sales, item, date_dim - WHERE ss_item_sk IN (SELECT ss_item_sk - FROM cross_items) - AND ss_item_sk = i_item_sk - AND ss_sold_date_sk = d_date_sk - AND d_week_seq = (SELECT d_week_seq - FROM date_dim - WHERE d_year = 1999 + 1 AND d_moy = 12 AND d_dom = 11) - GROUP BY i_brand_id, i_class_id, i_category_id - HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales - FROM avg_sales)) this_year, - (SELECT - 'store' channel, - i_brand_id, - i_class_id, - i_category_id, - sum(ss_quantity * ss_list_price) sales, - count(*) number_sales - FROM store_sales, item, date_dim - WHERE ss_item_sk IN (SELECT ss_item_sk - FROM cross_items) - AND ss_item_sk = i_item_sk - AND ss_sold_date_sk = d_date_sk - AND d_week_seq = (SELECT d_week_seq - FROM date_dim - WHERE d_year = 1999 AND d_moy = 12 AND d_dom = 11) - GROUP BY i_brand_id, i_class_id, i_category_id - HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales - FROM avg_sales)) last_year -WHERE this_year.i_brand_id = last_year.i_brand_id - AND this_year.i_class_id = last_year.i_class_id - AND this_year.i_category_id = last_year.i_category_id -ORDER BY this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q15.sql b/src/test/resources/tpcds/queries/q15.sql deleted file mode 100644 index b8182e23b..000000000 --- a/src/test/resources/tpcds/queries/q15.sql +++ /dev/null @@ -1,15 +0,0 @@ -SELECT - ca_zip, - sum(cs_sales_price) -FROM catalog_sales, customer, customer_address, date_dim -WHERE cs_bill_customer_sk = c_customer_sk - AND c_current_addr_sk = ca_address_sk - AND (substr(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', - '85392', '85460', '80348', '81792') - OR ca_state IN ('CA', 'WA', 'GA') - OR cs_sales_price > 500) - AND cs_sold_date_sk = d_date_sk - AND d_qoy = 2 AND d_year = 2001 -GROUP BY ca_zip -ORDER BY ca_zip -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q16.sql b/src/test/resources/tpcds/queries/q16.sql deleted file mode 100644 index 732ad0d84..000000000 --- a/src/test/resources/tpcds/queries/q16.sql +++ /dev/null @@ -1,23 +0,0 @@ -SELECT - count(DISTINCT cs_order_number) AS `order count `, - sum(cs_ext_ship_cost) AS `total shipping cost `, - sum(cs_net_profit) AS `total net profit ` -FROM - catalog_sales cs1, date_dim, customer_address, call_center -WHERE - d_date BETWEEN '2002-02-01' AND (CAST('2002-02-01' AS DATE) + INTERVAL 60 days) - AND cs1.cs_ship_date_sk = d_date_sk - AND cs1.cs_ship_addr_sk = ca_address_sk - AND ca_state = 'GA' - AND cs1.cs_call_center_sk = cc_call_center_sk - AND cc_county IN - ('Williamson County', 'Williamson County', 'Williamson County', 'Williamson County', 'Williamson County') - AND EXISTS(SELECT * - FROM catalog_sales cs2 - WHERE cs1.cs_order_number = cs2.cs_order_number - AND cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) - AND NOT EXISTS(SELECT * - FROM catalog_returns cr1 - WHERE cs1.cs_order_number = cr1.cr_order_number) -ORDER BY count(DISTINCT cs_order_number) -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q17.sql b/src/test/resources/tpcds/queries/q17.sql deleted file mode 100644 index 4d647f795..000000000 --- a/src/test/resources/tpcds/queries/q17.sql +++ /dev/null @@ -1,33 +0,0 @@ -SELECT - i_item_id, - i_item_desc, - s_state, - count(ss_quantity) AS store_sales_quantitycount, - avg(ss_quantity) AS store_sales_quantityave, - stddev_samp(ss_quantity) AS store_sales_quantitystdev, - stddev_samp(ss_quantity) / avg(ss_quantity) AS store_sales_quantitycov, - count(sr_return_quantity) as_store_returns_quantitycount, - avg(sr_return_quantity) as_store_returns_quantityave, - stddev_samp(sr_return_quantity) as_store_returns_quantitystdev, - stddev_samp(sr_return_quantity) / avg(sr_return_quantity) AS store_returns_quantitycov, - count(cs_quantity) AS catalog_sales_quantitycount, - avg(cs_quantity) AS catalog_sales_quantityave, - stddev_samp(cs_quantity) / avg(cs_quantity) AS catalog_sales_quantitystdev, - stddev_samp(cs_quantity) / avg(cs_quantity) AS catalog_sales_quantitycov -FROM store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, store, item -WHERE d1.d_quarter_name = '2001Q1' - AND d1.d_date_sk = ss_sold_date_sk - AND i_item_sk = ss_item_sk - AND s_store_sk = ss_store_sk - AND ss_customer_sk = sr_customer_sk - AND ss_item_sk = sr_item_sk - AND ss_ticket_number = sr_ticket_number - AND sr_returned_date_sk = d2.d_date_sk - AND d2.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') - AND sr_customer_sk = cs_bill_customer_sk - AND sr_item_sk = cs_item_sk - AND cs_sold_date_sk = d3.d_date_sk - AND d3.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') -GROUP BY i_item_id, i_item_desc, s_state -ORDER BY i_item_id, i_item_desc, s_state -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q18.sql b/src/test/resources/tpcds/queries/q18.sql deleted file mode 100644 index 4055c80fd..000000000 --- a/src/test/resources/tpcds/queries/q18.sql +++ /dev/null @@ -1,28 +0,0 @@ -SELECT - i_item_id, - ca_country, - ca_state, - ca_county, - avg(cast(cs_quantity AS DECIMAL(12, 2))) agg1, - avg(cast(cs_list_price AS DECIMAL(12, 2))) agg2, - avg(cast(cs_coupon_amt AS DECIMAL(12, 2))) agg3, - avg(cast(cs_sales_price AS DECIMAL(12, 2))) agg4, - avg(cast(cs_net_profit AS DECIMAL(12, 2))) agg5, - avg(cast(c_birth_year AS DECIMAL(12, 2))) agg6, - avg(cast(cd1.cd_dep_count AS DECIMAL(12, 2))) agg7 -FROM catalog_sales, customer_demographics cd1, - customer_demographics cd2, customer, customer_address, date_dim, item -WHERE cs_sold_date_sk = d_date_sk AND - cs_item_sk = i_item_sk AND - cs_bill_cdemo_sk = cd1.cd_demo_sk AND - cs_bill_customer_sk = c_customer_sk AND - cd1.cd_gender = 'F' AND - cd1.cd_education_status = 'Unknown' AND - c_current_cdemo_sk = cd2.cd_demo_sk AND - c_current_addr_sk = ca_address_sk AND - c_birth_month IN (1, 6, 8, 9, 12, 2) AND - d_year = 1998 AND - ca_state IN ('MS', 'IN', 'ND', 'OK', 'NM', 'VA', 'MS') -GROUP BY ROLLUP (i_item_id, ca_country, ca_state, ca_county) -ORDER BY ca_country, ca_state, ca_county, i_item_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q19.sql b/src/test/resources/tpcds/queries/q19.sql deleted file mode 100644 index e38ab7f26..000000000 --- a/src/test/resources/tpcds/queries/q19.sql +++ /dev/null @@ -1,19 +0,0 @@ -SELECT - i_brand_id brand_id, - i_brand brand, - i_manufact_id, - i_manufact, - sum(ss_ext_sales_price) ext_price -FROM date_dim, store_sales, item, customer, customer_address, store -WHERE d_date_sk = ss_sold_date_sk - AND ss_item_sk = i_item_sk - AND i_manager_id = 8 - AND d_moy = 11 - AND d_year = 1998 - AND ss_customer_sk = c_customer_sk - AND c_current_addr_sk = ca_address_sk - AND substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5) - AND ss_store_sk = s_store_sk -GROUP BY i_brand, i_brand_id, i_manufact_id, i_manufact -ORDER BY ext_price DESC, brand, brand_id, i_manufact_id, i_manufact -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q2.sql b/src/test/resources/tpcds/queries/q2.sql deleted file mode 100644 index 52c0e90c4..000000000 --- a/src/test/resources/tpcds/queries/q2.sql +++ /dev/null @@ -1,81 +0,0 @@ -WITH wscs AS -( SELECT - sold_date_sk, - sales_price - FROM (SELECT - ws_sold_date_sk sold_date_sk, - ws_ext_sales_price sales_price - FROM web_sales) x - UNION ALL - (SELECT - cs_sold_date_sk sold_date_sk, - cs_ext_sales_price sales_price - FROM catalog_sales)), - wswscs AS - ( SELECT - d_week_seq, - sum(CASE WHEN (d_day_name = 'Sunday') - THEN sales_price - ELSE NULL END) - sun_sales, - sum(CASE WHEN (d_day_name = 'Monday') - THEN sales_price - ELSE NULL END) - mon_sales, - sum(CASE WHEN (d_day_name = 'Tuesday') - THEN sales_price - ELSE NULL END) - tue_sales, - sum(CASE WHEN (d_day_name = 'Wednesday') - THEN sales_price - ELSE NULL END) - wed_sales, - sum(CASE WHEN (d_day_name = 'Thursday') - THEN sales_price - ELSE NULL END) - thu_sales, - sum(CASE WHEN (d_day_name = 'Friday') - THEN sales_price - ELSE NULL END) - fri_sales, - sum(CASE WHEN (d_day_name = 'Saturday') - THEN sales_price - ELSE NULL END) - sat_sales - FROM wscs, date_dim - WHERE d_date_sk = sold_date_sk - GROUP BY d_week_seq) -SELECT - d_week_seq1, - round(sun_sales1 / sun_sales2, 2), - round(mon_sales1 / mon_sales2, 2), - round(tue_sales1 / tue_sales2, 2), - round(wed_sales1 / wed_sales2, 2), - round(thu_sales1 / thu_sales2, 2), - round(fri_sales1 / fri_sales2, 2), - round(sat_sales1 / sat_sales2, 2) -FROM - (SELECT - wswscs.d_week_seq d_week_seq1, - sun_sales sun_sales1, - mon_sales mon_sales1, - tue_sales tue_sales1, - wed_sales wed_sales1, - thu_sales thu_sales1, - fri_sales fri_sales1, - sat_sales sat_sales1 - FROM wswscs, date_dim - WHERE date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001) y, - (SELECT - wswscs.d_week_seq d_week_seq2, - sun_sales sun_sales2, - mon_sales mon_sales2, - tue_sales tue_sales2, - wed_sales wed_sales2, - thu_sales thu_sales2, - fri_sales fri_sales2, - sat_sales sat_sales2 - FROM wswscs, date_dim - WHERE date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 + 1) z -WHERE d_week_seq1 = d_week_seq2 - 53 -ORDER BY d_week_seq1 diff --git a/src/test/resources/tpcds/queries/q20.sql b/src/test/resources/tpcds/queries/q20.sql deleted file mode 100644 index 7ac6c7a75..000000000 --- a/src/test/resources/tpcds/queries/q20.sql +++ /dev/null @@ -1,18 +0,0 @@ -SELECT - i_item_desc, - i_category, - i_class, - i_current_price, - sum(cs_ext_sales_price) AS itemrevenue, - sum(cs_ext_sales_price) * 100 / sum(sum(cs_ext_sales_price)) - OVER - (PARTITION BY i_class) AS revenueratio -FROM catalog_sales, item, date_dim -WHERE cs_item_sk = i_item_sk - AND i_category IN ('Sports', 'Books', 'Home') - AND cs_sold_date_sk = d_date_sk - AND d_date BETWEEN cast('1999-02-22' AS DATE) -AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) -GROUP BY i_item_id, i_item_desc, i_category, i_class, i_current_price -ORDER BY i_category, i_class, i_item_id, i_item_desc, revenueratio -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q21.sql b/src/test/resources/tpcds/queries/q21.sql deleted file mode 100644 index 550881143..000000000 --- a/src/test/resources/tpcds/queries/q21.sql +++ /dev/null @@ -1,25 +0,0 @@ -SELECT * -FROM ( - SELECT - w_warehouse_name, - i_item_id, - sum(CASE WHEN (cast(d_date AS DATE) < cast('2000-03-11' AS DATE)) - THEN inv_quantity_on_hand - ELSE 0 END) AS inv_before, - sum(CASE WHEN (cast(d_date AS DATE) >= cast('2000-03-11' AS DATE)) - THEN inv_quantity_on_hand - ELSE 0 END) AS inv_after - FROM inventory, warehouse, item, date_dim - WHERE i_current_price BETWEEN 0.99 AND 1.49 - AND i_item_sk = inv_item_sk - AND inv_warehouse_sk = w_warehouse_sk - AND inv_date_sk = d_date_sk - AND d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) - AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days) - GROUP BY w_warehouse_name, i_item_id) x -WHERE (CASE WHEN inv_before > 0 - THEN inv_after / inv_before - ELSE NULL - END) BETWEEN 2.0 / 3.0 AND 3.0 / 2.0 -ORDER BY w_warehouse_name, i_item_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q22.sql b/src/test/resources/tpcds/queries/q22.sql deleted file mode 100644 index add3b41f7..000000000 --- a/src/test/resources/tpcds/queries/q22.sql +++ /dev/null @@ -1,14 +0,0 @@ -SELECT - i_product_name, - i_brand, - i_class, - i_category, - avg(inv_quantity_on_hand) qoh -FROM inventory, date_dim, item, warehouse -WHERE inv_date_sk = d_date_sk - AND inv_item_sk = i_item_sk - AND inv_warehouse_sk = w_warehouse_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11 -GROUP BY ROLLUP (i_product_name, i_brand, i_class, i_category) -ORDER BY qoh, i_product_name, i_brand, i_class, i_category -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q23a.sql b/src/test/resources/tpcds/queries/q23a.sql deleted file mode 100644 index 37791f643..000000000 --- a/src/test/resources/tpcds/queries/q23a.sql +++ /dev/null @@ -1,53 +0,0 @@ -WITH frequent_ss_items AS -(SELECT - substr(i_item_desc, 1, 30) itemdesc, - i_item_sk item_sk, - d_date solddate, - count(*) cnt - FROM store_sales, date_dim, item - WHERE ss_sold_date_sk = d_date_sk - AND ss_item_sk = i_item_sk - AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) - GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date - HAVING count(*) > 4), - max_store_sales AS - (SELECT max(csales) tpcds_cmax - FROM (SELECT - c_customer_sk, - sum(ss_quantity * ss_sales_price) csales - FROM store_sales, customer, date_dim - WHERE ss_customer_sk = c_customer_sk - AND ss_sold_date_sk = d_date_sk - AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) - GROUP BY c_customer_sk) x), - best_ss_customer AS - (SELECT - c_customer_sk, - sum(ss_quantity * ss_sales_price) ssales - FROM store_sales, customer - WHERE ss_customer_sk = c_customer_sk - GROUP BY c_customer_sk - HAVING sum(ss_quantity * ss_sales_price) > (50 / 100.0) * - (SELECT * - FROM max_store_sales)) -SELECT sum(sales) -FROM ((SELECT cs_quantity * cs_list_price sales -FROM catalog_sales, date_dim -WHERE d_year = 2000 - AND d_moy = 2 - AND cs_sold_date_sk = d_date_sk - AND cs_item_sk IN (SELECT item_sk -FROM frequent_ss_items) - AND cs_bill_customer_sk IN (SELECT c_customer_sk -FROM best_ss_customer)) - UNION ALL - (SELECT ws_quantity * ws_list_price sales - FROM web_sales, date_dim - WHERE d_year = 2000 - AND d_moy = 2 - AND ws_sold_date_sk = d_date_sk - AND ws_item_sk IN (SELECT item_sk - FROM frequent_ss_items) - AND ws_bill_customer_sk IN (SELECT c_customer_sk - FROM best_ss_customer))) y -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q23b.sql b/src/test/resources/tpcds/queries/q23b.sql deleted file mode 100644 index 01150197a..000000000 --- a/src/test/resources/tpcds/queries/q23b.sql +++ /dev/null @@ -1,68 +0,0 @@ -WITH frequent_ss_items AS -(SELECT - substr(i_item_desc, 1, 30) itemdesc, - i_item_sk item_sk, - d_date solddate, - count(*) cnt - FROM store_sales, date_dim, item - WHERE ss_sold_date_sk = d_date_sk - AND ss_item_sk = i_item_sk - AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) - GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date - HAVING count(*) > 4), - max_store_sales AS - (SELECT max(csales) tpcds_cmax - FROM (SELECT - c_customer_sk, - sum(ss_quantity * ss_sales_price) csales - FROM store_sales, customer, date_dim - WHERE ss_customer_sk = c_customer_sk - AND ss_sold_date_sk = d_date_sk - AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) - GROUP BY c_customer_sk) x), - best_ss_customer AS - (SELECT - c_customer_sk, - sum(ss_quantity * ss_sales_price) ssales - FROM store_sales - , customer - WHERE ss_customer_sk = c_customer_sk - GROUP BY c_customer_sk - HAVING sum(ss_quantity * ss_sales_price) > (50 / 100.0) * - (SELECT * - FROM max_store_sales)) -SELECT - c_last_name, - c_first_name, - sales -FROM ((SELECT - c_last_name, - c_first_name, - sum(cs_quantity * cs_list_price) sales -FROM catalog_sales, customer, date_dim -WHERE d_year = 2000 - AND d_moy = 2 - AND cs_sold_date_sk = d_date_sk - AND cs_item_sk IN (SELECT item_sk -FROM frequent_ss_items) - AND cs_bill_customer_sk IN (SELECT c_customer_sk -FROM best_ss_customer) - AND cs_bill_customer_sk = c_customer_sk -GROUP BY c_last_name, c_first_name) - UNION ALL - (SELECT - c_last_name, - c_first_name, - sum(ws_quantity * ws_list_price) sales - FROM web_sales, customer, date_dim - WHERE d_year = 2000 - AND d_moy = 2 - AND ws_sold_date_sk = d_date_sk - AND ws_item_sk IN (SELECT item_sk - FROM frequent_ss_items) - AND ws_bill_customer_sk IN (SELECT c_customer_sk - FROM best_ss_customer) - AND ws_bill_customer_sk = c_customer_sk - GROUP BY c_last_name, c_first_name)) y -ORDER BY c_last_name, c_first_name, sales -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q24a.sql b/src/test/resources/tpcds/queries/q24a.sql deleted file mode 100644 index bcc189486..000000000 --- a/src/test/resources/tpcds/queries/q24a.sql +++ /dev/null @@ -1,34 +0,0 @@ -WITH ssales AS -(SELECT - c_last_name, - c_first_name, - s_store_name, - ca_state, - s_state, - i_color, - i_current_price, - i_manager_id, - i_units, - i_size, - sum(ss_net_paid) netpaid - FROM store_sales, store_returns, store, item, customer, customer_address - WHERE ss_ticket_number = sr_ticket_number - AND ss_item_sk = sr_item_sk - AND ss_customer_sk = c_customer_sk - AND ss_item_sk = i_item_sk - AND ss_store_sk = s_store_sk - AND c_birth_country = upper(ca_country) - AND s_zip = ca_zip - AND s_market_id = 8 - GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, - i_current_price, i_manager_id, i_units, i_size) -SELECT - c_last_name, - c_first_name, - s_store_name, - sum(netpaid) paid -FROM ssales -WHERE i_color = 'pale' -GROUP BY c_last_name, c_first_name, s_store_name -HAVING sum(netpaid) > (SELECT 0.05 * avg(netpaid) -FROM ssales) diff --git a/src/test/resources/tpcds/queries/q24b.sql b/src/test/resources/tpcds/queries/q24b.sql deleted file mode 100644 index 830eb670b..000000000 --- a/src/test/resources/tpcds/queries/q24b.sql +++ /dev/null @@ -1,34 +0,0 @@ -WITH ssales AS -(SELECT - c_last_name, - c_first_name, - s_store_name, - ca_state, - s_state, - i_color, - i_current_price, - i_manager_id, - i_units, - i_size, - sum(ss_net_paid) netpaid - FROM store_sales, store_returns, store, item, customer, customer_address - WHERE ss_ticket_number = sr_ticket_number - AND ss_item_sk = sr_item_sk - AND ss_customer_sk = c_customer_sk - AND ss_item_sk = i_item_sk - AND ss_store_sk = s_store_sk - AND c_birth_country = upper(ca_country) - AND s_zip = ca_zip - AND s_market_id = 8 - GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, - i_color, i_current_price, i_manager_id, i_units, i_size) -SELECT - c_last_name, - c_first_name, - s_store_name, - sum(netpaid) paid -FROM ssales -WHERE i_color = 'chiffon' -GROUP BY c_last_name, c_first_name, s_store_name -HAVING sum(netpaid) > (SELECT 0.05 * avg(netpaid) -FROM ssales) diff --git a/src/test/resources/tpcds/queries/q25.sql b/src/test/resources/tpcds/queries/q25.sql deleted file mode 100644 index a4d78a3c5..000000000 --- a/src/test/resources/tpcds/queries/q25.sql +++ /dev/null @@ -1,33 +0,0 @@ -SELECT - i_item_id, - i_item_desc, - s_store_id, - s_store_name, - sum(ss_net_profit) AS store_sales_profit, - sum(sr_net_loss) AS store_returns_loss, - sum(cs_net_profit) AS catalog_sales_profit -FROM - store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, - store, item -WHERE - d1.d_moy = 4 - AND d1.d_year = 2001 - AND d1.d_date_sk = ss_sold_date_sk - AND i_item_sk = ss_item_sk - AND s_store_sk = ss_store_sk - AND ss_customer_sk = sr_customer_sk - AND ss_item_sk = sr_item_sk - AND ss_ticket_number = sr_ticket_number - AND sr_returned_date_sk = d2.d_date_sk - AND d2.d_moy BETWEEN 4 AND 10 - AND d2.d_year = 2001 - AND sr_customer_sk = cs_bill_customer_sk - AND sr_item_sk = cs_item_sk - AND cs_sold_date_sk = d3.d_date_sk - AND d3.d_moy BETWEEN 4 AND 10 - AND d3.d_year = 2001 -GROUP BY - i_item_id, i_item_desc, s_store_id, s_store_name -ORDER BY - i_item_id, i_item_desc, s_store_id, s_store_name -LIMIT 100 \ No newline at end of file diff --git a/src/test/resources/tpcds/queries/q26.sql b/src/test/resources/tpcds/queries/q26.sql deleted file mode 100644 index 6d395a1d7..000000000 --- a/src/test/resources/tpcds/queries/q26.sql +++ /dev/null @@ -1,19 +0,0 @@ -SELECT - i_item_id, - avg(cs_quantity) agg1, - avg(cs_list_price) agg2, - avg(cs_coupon_amt) agg3, - avg(cs_sales_price) agg4 -FROM catalog_sales, customer_demographics, date_dim, item, promotion -WHERE cs_sold_date_sk = d_date_sk AND - cs_item_sk = i_item_sk AND - cs_bill_cdemo_sk = cd_demo_sk AND - cs_promo_sk = p_promo_sk AND - cd_gender = 'M' AND - cd_marital_status = 'S' AND - cd_education_status = 'College' AND - (p_channel_email = 'N' OR p_channel_event = 'N') AND - d_year = 2000 -GROUP BY i_item_id -ORDER BY i_item_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q27.sql b/src/test/resources/tpcds/queries/q27.sql deleted file mode 100644 index b0e2fd95f..000000000 --- a/src/test/resources/tpcds/queries/q27.sql +++ /dev/null @@ -1,21 +0,0 @@ -SELECT - i_item_id, - s_state, - grouping(s_state) g_state, - avg(ss_quantity) agg1, - avg(ss_list_price) agg2, - avg(ss_coupon_amt) agg3, - avg(ss_sales_price) agg4 -FROM store_sales, customer_demographics, date_dim, store, item -WHERE ss_sold_date_sk = d_date_sk AND - ss_item_sk = i_item_sk AND - ss_store_sk = s_store_sk AND - ss_cdemo_sk = cd_demo_sk AND - cd_gender = 'M' AND - cd_marital_status = 'S' AND - cd_education_status = 'College' AND - d_year = 2002 AND - s_state IN ('TN', 'TN', 'TN', 'TN', 'TN', 'TN') -GROUP BY ROLLUP (i_item_id, s_state) -ORDER BY i_item_id, s_state -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q28.sql b/src/test/resources/tpcds/queries/q28.sql deleted file mode 100644 index f34c2bb0e..000000000 --- a/src/test/resources/tpcds/queries/q28.sql +++ /dev/null @@ -1,56 +0,0 @@ -SELECT * -FROM (SELECT - avg(ss_list_price) B1_LP, - count(ss_list_price) B1_CNT, - count(DISTINCT ss_list_price) B1_CNTD -FROM store_sales -WHERE ss_quantity BETWEEN 0 AND 5 - AND (ss_list_price BETWEEN 8 AND 8 + 10 - OR ss_coupon_amt BETWEEN 459 AND 459 + 1000 - OR ss_wholesale_cost BETWEEN 57 AND 57 + 20)) B1, - (SELECT - avg(ss_list_price) B2_LP, - count(ss_list_price) B2_CNT, - count(DISTINCT ss_list_price) B2_CNTD - FROM store_sales - WHERE ss_quantity BETWEEN 6 AND 10 - AND (ss_list_price BETWEEN 90 AND 90 + 10 - OR ss_coupon_amt BETWEEN 2323 AND 2323 + 1000 - OR ss_wholesale_cost BETWEEN 31 AND 31 + 20)) B2, - (SELECT - avg(ss_list_price) B3_LP, - count(ss_list_price) B3_CNT, - count(DISTINCT ss_list_price) B3_CNTD - FROM store_sales - WHERE ss_quantity BETWEEN 11 AND 15 - AND (ss_list_price BETWEEN 142 AND 142 + 10 - OR ss_coupon_amt BETWEEN 12214 AND 12214 + 1000 - OR ss_wholesale_cost BETWEEN 79 AND 79 + 20)) B3, - (SELECT - avg(ss_list_price) B4_LP, - count(ss_list_price) B4_CNT, - count(DISTINCT ss_list_price) B4_CNTD - FROM store_sales - WHERE ss_quantity BETWEEN 16 AND 20 - AND (ss_list_price BETWEEN 135 AND 135 + 10 - OR ss_coupon_amt BETWEEN 6071 AND 6071 + 1000 - OR ss_wholesale_cost BETWEEN 38 AND 38 + 20)) B4, - (SELECT - avg(ss_list_price) B5_LP, - count(ss_list_price) B5_CNT, - count(DISTINCT ss_list_price) B5_CNTD - FROM store_sales - WHERE ss_quantity BETWEEN 21 AND 25 - AND (ss_list_price BETWEEN 122 AND 122 + 10 - OR ss_coupon_amt BETWEEN 836 AND 836 + 1000 - OR ss_wholesale_cost BETWEEN 17 AND 17 + 20)) B5, - (SELECT - avg(ss_list_price) B6_LP, - count(ss_list_price) B6_CNT, - count(DISTINCT ss_list_price) B6_CNTD - FROM store_sales - WHERE ss_quantity BETWEEN 26 AND 30 - AND (ss_list_price BETWEEN 154 AND 154 + 10 - OR ss_coupon_amt BETWEEN 7326 AND 7326 + 1000 - OR ss_wholesale_cost BETWEEN 7 AND 7 + 20)) B6 -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q29.sql b/src/test/resources/tpcds/queries/q29.sql deleted file mode 100644 index 3f1fd553f..000000000 --- a/src/test/resources/tpcds/queries/q29.sql +++ /dev/null @@ -1,32 +0,0 @@ -SELECT - i_item_id, - i_item_desc, - s_store_id, - s_store_name, - sum(ss_quantity) AS store_sales_quantity, - sum(sr_return_quantity) AS store_returns_quantity, - sum(cs_quantity) AS catalog_sales_quantity -FROM - store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, - date_dim d3, store, item -WHERE - d1.d_moy = 9 - AND d1.d_year = 1999 - AND d1.d_date_sk = ss_sold_date_sk - AND i_item_sk = ss_item_sk - AND s_store_sk = ss_store_sk - AND ss_customer_sk = sr_customer_sk - AND ss_item_sk = sr_item_sk - AND ss_ticket_number = sr_ticket_number - AND sr_returned_date_sk = d2.d_date_sk - AND d2.d_moy BETWEEN 9 AND 9 + 3 - AND d2.d_year = 1999 - AND sr_customer_sk = cs_bill_customer_sk - AND sr_item_sk = cs_item_sk - AND cs_sold_date_sk = d3.d_date_sk - AND d3.d_year IN (1999, 1999 + 1, 1999 + 2) -GROUP BY - i_item_id, i_item_desc, s_store_id, s_store_name -ORDER BY - i_item_id, i_item_desc, s_store_id, s_store_name -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q3.sql b/src/test/resources/tpcds/queries/q3.sql deleted file mode 100644 index 181509df9..000000000 --- a/src/test/resources/tpcds/queries/q3.sql +++ /dev/null @@ -1,13 +0,0 @@ -SELECT - dt.d_year, - item.i_brand_id brand_id, - item.i_brand brand, - SUM(ss_ext_sales_price) sum_agg -FROM date_dim dt, store_sales, item -WHERE dt.d_date_sk = store_sales.ss_sold_date_sk - AND store_sales.ss_item_sk = item.i_item_sk - AND item.i_manufact_id = 128 - AND dt.d_moy = 11 -GROUP BY dt.d_year, item.i_brand, item.i_brand_id -ORDER BY dt.d_year, sum_agg DESC, brand_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q30.sql b/src/test/resources/tpcds/queries/q30.sql deleted file mode 100644 index 986bef566..000000000 --- a/src/test/resources/tpcds/queries/q30.sql +++ /dev/null @@ -1,35 +0,0 @@ -WITH customer_total_return AS -(SELECT - wr_returning_customer_sk AS ctr_customer_sk, - ca_state AS ctr_state, - sum(wr_return_amt) AS ctr_total_return - FROM web_returns, date_dim, customer_address - WHERE wr_returned_date_sk = d_date_sk - AND d_year = 2002 - AND wr_returning_addr_sk = ca_address_sk - GROUP BY wr_returning_customer_sk, ca_state) -SELECT - c_customer_id, - c_salutation, - c_first_name, - c_last_name, - c_preferred_cust_flag, - c_birth_day, - c_birth_month, - c_birth_year, - c_birth_country, - c_login, - c_email_address, - c_last_review_date, - ctr_total_return -FROM customer_total_return ctr1, customer_address, customer -WHERE ctr1.ctr_total_return > (SELECT avg(ctr_total_return) * 1.2 -FROM customer_total_return ctr2 -WHERE ctr1.ctr_state = ctr2.ctr_state) - AND ca_address_sk = c_current_addr_sk - AND ca_state = 'GA' - AND ctr1.ctr_customer_sk = c_customer_sk -ORDER BY c_customer_id, c_salutation, c_first_name, c_last_name, c_preferred_cust_flag - , c_birth_day, c_birth_month, c_birth_year, c_birth_country, c_login, c_email_address - , c_last_review_date, ctr_total_return -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q31.sql b/src/test/resources/tpcds/queries/q31.sql deleted file mode 100644 index 3e543d543..000000000 --- a/src/test/resources/tpcds/queries/q31.sql +++ /dev/null @@ -1,60 +0,0 @@ -WITH ss AS -(SELECT - ca_county, - d_qoy, - d_year, - sum(ss_ext_sales_price) AS store_sales - FROM store_sales, date_dim, customer_address - WHERE ss_sold_date_sk = d_date_sk - AND ss_addr_sk = ca_address_sk - GROUP BY ca_county, d_qoy, d_year), - ws AS - (SELECT - ca_county, - d_qoy, - d_year, - sum(ws_ext_sales_price) AS web_sales - FROM web_sales, date_dim, customer_address - WHERE ws_sold_date_sk = d_date_sk - AND ws_bill_addr_sk = ca_address_sk - GROUP BY ca_county, d_qoy, d_year) -SELECT - ss1.ca_county, - ss1.d_year, - ws2.web_sales / ws1.web_sales web_q1_q2_increase, - ss2.store_sales / ss1.store_sales store_q1_q2_increase, - ws3.web_sales / ws2.web_sales web_q2_q3_increase, - ss3.store_sales / ss2.store_sales store_q2_q3_increase -FROM - ss ss1, ss ss2, ss ss3, ws ws1, ws ws2, ws ws3 -WHERE - ss1.d_qoy = 1 - AND ss1.d_year = 2000 - AND ss1.ca_county = ss2.ca_county - AND ss2.d_qoy = 2 - AND ss2.d_year = 2000 - AND ss2.ca_county = ss3.ca_county - AND ss3.d_qoy = 3 - AND ss3.d_year = 2000 - AND ss1.ca_county = ws1.ca_county - AND ws1.d_qoy = 1 - AND ws1.d_year = 2000 - AND ws1.ca_county = ws2.ca_county - AND ws2.d_qoy = 2 - AND ws2.d_year = 2000 - AND ws1.ca_county = ws3.ca_county - AND ws3.d_qoy = 3 - AND ws3.d_year = 2000 - AND CASE WHEN ws1.web_sales > 0 - THEN ws2.web_sales / ws1.web_sales - ELSE NULL END - > CASE WHEN ss1.store_sales > 0 - THEN ss2.store_sales / ss1.store_sales - ELSE NULL END - AND CASE WHEN ws2.web_sales > 0 - THEN ws3.web_sales / ws2.web_sales - ELSE NULL END - > CASE WHEN ss2.store_sales > 0 - THEN ss3.store_sales / ss2.store_sales - ELSE NULL END -ORDER BY ss1.ca_county diff --git a/src/test/resources/tpcds/queries/q32.sql b/src/test/resources/tpcds/queries/q32.sql deleted file mode 100644 index a6f59ecb8..000000000 --- a/src/test/resources/tpcds/queries/q32.sql +++ /dev/null @@ -1,15 +0,0 @@ -SELECT 1 AS `excess discount amount ` -FROM - catalog_sales, item, date_dim -WHERE - i_manufact_id = 977 - AND i_item_sk = cs_item_sk - AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + interval 90 days) - AND d_date_sk = cs_sold_date_sk - AND cs_ext_discount_amt > ( - SELECT 1.3 * avg(cs_ext_discount_amt) - FROM catalog_sales, date_dim - WHERE cs_item_sk = i_item_sk - AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + interval 90 days) - AND d_date_sk = cs_sold_date_sk) -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q33.sql b/src/test/resources/tpcds/queries/q33.sql deleted file mode 100644 index d24856aa5..000000000 --- a/src/test/resources/tpcds/queries/q33.sql +++ /dev/null @@ -1,65 +0,0 @@ -WITH ss AS ( - SELECT - i_manufact_id, - sum(ss_ext_sales_price) total_sales - FROM - store_sales, date_dim, customer_address, item - WHERE - i_manufact_id IN (SELECT i_manufact_id - FROM item - WHERE i_category IN ('Electronics')) - AND ss_item_sk = i_item_sk - AND ss_sold_date_sk = d_date_sk - AND d_year = 1998 - AND d_moy = 5 - AND ss_addr_sk = ca_address_sk - AND ca_gmt_offset = -5 - GROUP BY i_manufact_id), cs AS -(SELECT - i_manufact_id, - sum(cs_ext_sales_price) total_sales - FROM catalog_sales, date_dim, customer_address, item - WHERE - i_manufact_id IN ( - SELECT i_manufact_id - FROM item - WHERE - i_category IN ('Electronics')) - AND cs_item_sk = i_item_sk - AND cs_sold_date_sk = d_date_sk - AND d_year = 1998 - AND d_moy = 5 - AND cs_bill_addr_sk = ca_address_sk - AND ca_gmt_offset = -5 - GROUP BY i_manufact_id), - ws AS ( - SELECT - i_manufact_id, - sum(ws_ext_sales_price) total_sales - FROM - web_sales, date_dim, customer_address, item - WHERE - i_manufact_id IN (SELECT i_manufact_id - FROM item - WHERE i_category IN ('Electronics')) - AND ws_item_sk = i_item_sk - AND ws_sold_date_sk = d_date_sk - AND d_year = 1998 - AND d_moy = 5 - AND ws_bill_addr_sk = ca_address_sk - AND ca_gmt_offset = -5 - GROUP BY i_manufact_id) -SELECT - i_manufact_id, - sum(total_sales) total_sales -FROM (SELECT * - FROM ss - UNION ALL - SELECT * - FROM cs - UNION ALL - SELECT * - FROM ws) tmp1 -GROUP BY i_manufact_id -ORDER BY total_sales -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q34.sql b/src/test/resources/tpcds/queries/q34.sql deleted file mode 100644 index 33396bf16..000000000 --- a/src/test/resources/tpcds/queries/q34.sql +++ /dev/null @@ -1,32 +0,0 @@ -SELECT - c_last_name, - c_first_name, - c_salutation, - c_preferred_cust_flag, - ss_ticket_number, - cnt -FROM - (SELECT - ss_ticket_number, - ss_customer_sk, - count(*) cnt - FROM store_sales, date_dim, store, household_demographics - WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk - AND store_sales.ss_store_sk = store.s_store_sk - AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk - AND (date_dim.d_dom BETWEEN 1 AND 3 OR date_dim.d_dom BETWEEN 25 AND 28) - AND (household_demographics.hd_buy_potential = '>10000' OR - household_demographics.hd_buy_potential = 'unknown') - AND household_demographics.hd_vehicle_count > 0 - AND (CASE WHEN household_demographics.hd_vehicle_count > 0 - THEN household_demographics.hd_dep_count / household_demographics.hd_vehicle_count - ELSE NULL - END) > 1.2 - AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) - AND store.s_county IN - ('Williamson County', 'Williamson County', 'Williamson County', 'Williamson County', - 'Williamson County', 'Williamson County', 'Williamson County', 'Williamson County') - GROUP BY ss_ticket_number, ss_customer_sk) dn, customer -WHERE ss_customer_sk = c_customer_sk - AND cnt BETWEEN 15 AND 20 -ORDER BY c_last_name, c_first_name, c_salutation, c_preferred_cust_flag DESC diff --git a/src/test/resources/tpcds/queries/q35.sql b/src/test/resources/tpcds/queries/q35.sql deleted file mode 100644 index cfe4342d8..000000000 --- a/src/test/resources/tpcds/queries/q35.sql +++ /dev/null @@ -1,46 +0,0 @@ -SELECT - ca_state, - cd_gender, - cd_marital_status, - count(*) cnt1, - min(cd_dep_count), - max(cd_dep_count), - avg(cd_dep_count), - cd_dep_employed_count, - count(*) cnt2, - min(cd_dep_employed_count), - max(cd_dep_employed_count), - avg(cd_dep_employed_count), - cd_dep_college_count, - count(*) cnt3, - min(cd_dep_college_count), - max(cd_dep_college_count), - avg(cd_dep_college_count) -FROM - customer c, customer_address ca, customer_demographics -WHERE - c.c_current_addr_sk = ca.ca_address_sk AND - cd_demo_sk = c.c_current_cdemo_sk AND - exists(SELECT * - FROM store_sales, date_dim - WHERE c.c_customer_sk = ss_customer_sk AND - ss_sold_date_sk = d_date_sk AND - d_year = 2002 AND - d_qoy < 4) AND - (exists(SELECT * - FROM web_sales, date_dim - WHERE c.c_customer_sk = ws_bill_customer_sk AND - ws_sold_date_sk = d_date_sk AND - d_year = 2002 AND - d_qoy < 4) OR - exists(SELECT * - FROM catalog_sales, date_dim - WHERE c.c_customer_sk = cs_ship_customer_sk AND - cs_sold_date_sk = d_date_sk AND - d_year = 2002 AND - d_qoy < 4)) -GROUP BY ca_state, cd_gender, cd_marital_status, cd_dep_count, - cd_dep_employed_count, cd_dep_college_count -ORDER BY ca_state, cd_gender, cd_marital_status, cd_dep_count, - cd_dep_employed_count, cd_dep_college_count -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q36.sql b/src/test/resources/tpcds/queries/q36.sql deleted file mode 100644 index a8f93df76..000000000 --- a/src/test/resources/tpcds/queries/q36.sql +++ /dev/null @@ -1,26 +0,0 @@ -SELECT - sum(ss_net_profit) / sum(ss_ext_sales_price) AS gross_margin, - i_category, - i_class, - grouping(i_category) + grouping(i_class) AS lochierarchy, - rank() - OVER ( - PARTITION BY grouping(i_category) + grouping(i_class), - CASE WHEN grouping(i_class) = 0 - THEN i_category END - ORDER BY sum(ss_net_profit) / sum(ss_ext_sales_price) ASC) AS rank_within_parent -FROM - store_sales, date_dim d1, item, store -WHERE - d1.d_year = 2001 - AND d1.d_date_sk = ss_sold_date_sk - AND i_item_sk = ss_item_sk - AND s_store_sk = ss_store_sk - AND s_state IN ('TN', 'TN', 'TN', 'TN', 'TN', 'TN', 'TN', 'TN') -GROUP BY ROLLUP (i_category, i_class) -ORDER BY - lochierarchy DESC - , CASE WHEN lochierarchy = 0 - THEN i_category END - , rank_within_parent -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q37.sql b/src/test/resources/tpcds/queries/q37.sql deleted file mode 100644 index 11b3821fa..000000000 --- a/src/test/resources/tpcds/queries/q37.sql +++ /dev/null @@ -1,15 +0,0 @@ -SELECT - i_item_id, - i_item_desc, - i_current_price -FROM item, inventory, date_dim, catalog_sales -WHERE i_current_price BETWEEN 68 AND 68 + 30 - AND inv_item_sk = i_item_sk - AND d_date_sk = inv_date_sk - AND d_date BETWEEN cast('2000-02-01' AS DATE) AND (cast('2000-02-01' AS DATE) + INTERVAL 60 days) - AND i_manufact_id IN (677, 940, 694, 808) - AND inv_quantity_on_hand BETWEEN 100 AND 500 - AND cs_item_sk = i_item_sk -GROUP BY i_item_id, i_item_desc, i_current_price -ORDER BY i_item_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q38.sql b/src/test/resources/tpcds/queries/q38.sql deleted file mode 100644 index 1c8d53ee2..000000000 --- a/src/test/resources/tpcds/queries/q38.sql +++ /dev/null @@ -1,30 +0,0 @@ -SELECT count(*) -FROM ( - SELECT DISTINCT - c_last_name, - c_first_name, - d_date - FROM store_sales, date_dim, customer - WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk - AND store_sales.ss_customer_sk = customer.c_customer_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11 - INTERSECT - SELECT DISTINCT - c_last_name, - c_first_name, - d_date - FROM catalog_sales, date_dim, customer - WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk - AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11 - INTERSECT - SELECT DISTINCT - c_last_name, - c_first_name, - d_date - FROM web_sales, date_dim, customer - WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk - AND web_sales.ws_bill_customer_sk = customer.c_customer_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11 - ) hot_cust -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q39a.sql b/src/test/resources/tpcds/queries/q39a.sql deleted file mode 100644 index 9fc4c1701..000000000 --- a/src/test/resources/tpcds/queries/q39a.sql +++ /dev/null @@ -1,47 +0,0 @@ -WITH inv AS -(SELECT - w_warehouse_name, - w_warehouse_sk, - i_item_sk, - d_moy, - stdev, - mean, - CASE mean - WHEN 0 - THEN NULL - ELSE stdev / mean END cov - FROM (SELECT - w_warehouse_name, - w_warehouse_sk, - i_item_sk, - d_moy, - stddev_samp(inv_quantity_on_hand) stdev, - avg(inv_quantity_on_hand) mean - FROM inventory, item, warehouse, date_dim - WHERE inv_item_sk = i_item_sk - AND inv_warehouse_sk = w_warehouse_sk - AND inv_date_sk = d_date_sk - AND d_year = 2001 - GROUP BY w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo - WHERE CASE mean - WHEN 0 - THEN 0 - ELSE stdev / mean END > 1) -SELECT - inv1.w_warehouse_sk, - inv1.i_item_sk, - inv1.d_moy, - inv1.mean, - inv1.cov, - inv2.w_warehouse_sk, - inv2.i_item_sk, - inv2.d_moy, - inv2.mean, - inv2.cov -FROM inv inv1, inv inv2 -WHERE inv1.i_item_sk = inv2.i_item_sk - AND inv1.w_warehouse_sk = inv2.w_warehouse_sk - AND inv1.d_moy = 1 - AND inv2.d_moy = 1 + 1 -ORDER BY inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov - , inv2.d_moy, inv2.mean, inv2.cov diff --git a/src/test/resources/tpcds/queries/q39b.sql b/src/test/resources/tpcds/queries/q39b.sql deleted file mode 100644 index 6f8493029..000000000 --- a/src/test/resources/tpcds/queries/q39b.sql +++ /dev/null @@ -1,48 +0,0 @@ -WITH inv AS -(SELECT - w_warehouse_name, - w_warehouse_sk, - i_item_sk, - d_moy, - stdev, - mean, - CASE mean - WHEN 0 - THEN NULL - ELSE stdev / mean END cov - FROM (SELECT - w_warehouse_name, - w_warehouse_sk, - i_item_sk, - d_moy, - stddev_samp(inv_quantity_on_hand) stdev, - avg(inv_quantity_on_hand) mean - FROM inventory, item, warehouse, date_dim - WHERE inv_item_sk = i_item_sk - AND inv_warehouse_sk = w_warehouse_sk - AND inv_date_sk = d_date_sk - AND d_year = 2001 - GROUP BY w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo - WHERE CASE mean - WHEN 0 - THEN 0 - ELSE stdev / mean END > 1) -SELECT - inv1.w_warehouse_sk, - inv1.i_item_sk, - inv1.d_moy, - inv1.mean, - inv1.cov, - inv2.w_warehouse_sk, - inv2.i_item_sk, - inv2.d_moy, - inv2.mean, - inv2.cov -FROM inv inv1, inv inv2 -WHERE inv1.i_item_sk = inv2.i_item_sk - AND inv1.w_warehouse_sk = inv2.w_warehouse_sk - AND inv1.d_moy = 1 - AND inv2.d_moy = 1 + 1 - AND inv1.cov > 1.5 -ORDER BY inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov - , inv2.d_moy, inv2.mean, inv2.cov diff --git a/src/test/resources/tpcds/queries/q4.sql b/src/test/resources/tpcds/queries/q4.sql deleted file mode 100644 index b9f27fbc9..000000000 --- a/src/test/resources/tpcds/queries/q4.sql +++ /dev/null @@ -1,120 +0,0 @@ -WITH year_total AS ( - SELECT - c_customer_id customer_id, - c_first_name customer_first_name, - c_last_name customer_last_name, - c_preferred_cust_flag customer_preferred_cust_flag, - c_birth_country customer_birth_country, - c_login customer_login, - c_email_address customer_email_address, - d_year dyear, - sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + - ss_ext_sales_price) / 2) year_total, - 's' sale_type - FROM customer, store_sales, date_dim - WHERE c_customer_sk = ss_customer_sk AND ss_sold_date_sk = d_date_sk - GROUP BY c_customer_id, - c_first_name, - c_last_name, - c_preferred_cust_flag, - c_birth_country, - c_login, - c_email_address, - d_year - UNION ALL - SELECT - c_customer_id customer_id, - c_first_name customer_first_name, - c_last_name customer_last_name, - c_preferred_cust_flag customer_preferred_cust_flag, - c_birth_country customer_birth_country, - c_login customer_login, - c_email_address customer_email_address, - d_year dyear, - sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + - cs_ext_sales_price) / 2)) year_total, - 'c' sale_type - FROM customer, catalog_sales, date_dim - WHERE c_customer_sk = cs_bill_customer_sk AND cs_sold_date_sk = d_date_sk - GROUP BY c_customer_id, - c_first_name, - c_last_name, - c_preferred_cust_flag, - c_birth_country, - c_login, - c_email_address, - d_year - UNION ALL - SELECT - c_customer_id customer_id, - c_first_name customer_first_name, - c_last_name customer_last_name, - c_preferred_cust_flag customer_preferred_cust_flag, - c_birth_country customer_birth_country, - c_login customer_login, - c_email_address customer_email_address, - d_year dyear, - sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / - 2)) year_total, - 'w' sale_type - FROM customer, web_sales, date_dim - WHERE c_customer_sk = ws_bill_customer_sk AND ws_sold_date_sk = d_date_sk - GROUP BY c_customer_id, - c_first_name, - c_last_name, - c_preferred_cust_flag, - c_birth_country, - c_login, - c_email_address, - d_year) -SELECT - t_s_secyear.customer_id, - t_s_secyear.customer_first_name, - t_s_secyear.customer_last_name, - t_s_secyear.customer_preferred_cust_flag, - t_s_secyear.customer_birth_country, - t_s_secyear.customer_login, - t_s_secyear.customer_email_address -FROM year_total t_s_firstyear, year_total t_s_secyear, year_total t_c_firstyear, - year_total t_c_secyear, year_total t_w_firstyear, year_total t_w_secyear -WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id - AND t_s_firstyear.customer_id = t_c_secyear.customer_id - AND t_s_firstyear.customer_id = t_c_firstyear.customer_id - AND t_s_firstyear.customer_id = t_w_firstyear.customer_id - AND t_s_firstyear.customer_id = t_w_secyear.customer_id - AND t_s_firstyear.sale_type = 's' - AND t_c_firstyear.sale_type = 'c' - AND t_w_firstyear.sale_type = 'w' - AND t_s_secyear.sale_type = 's' - AND t_c_secyear.sale_type = 'c' - AND t_w_secyear.sale_type = 'w' - AND t_s_firstyear.dyear = 2001 - AND t_s_secyear.dyear = 2001 + 1 - AND t_c_firstyear.dyear = 2001 - AND t_c_secyear.dyear = 2001 + 1 - AND t_w_firstyear.dyear = 2001 - AND t_w_secyear.dyear = 2001 + 1 - AND t_s_firstyear.year_total > 0 - AND t_c_firstyear.year_total > 0 - AND t_w_firstyear.year_total > 0 - AND CASE WHEN t_c_firstyear.year_total > 0 - THEN t_c_secyear.year_total / t_c_firstyear.year_total - ELSE NULL END - > CASE WHEN t_s_firstyear.year_total > 0 - THEN t_s_secyear.year_total / t_s_firstyear.year_total - ELSE NULL END - AND CASE WHEN t_c_firstyear.year_total > 0 - THEN t_c_secyear.year_total / t_c_firstyear.year_total - ELSE NULL END - > CASE WHEN t_w_firstyear.year_total > 0 - THEN t_w_secyear.year_total / t_w_firstyear.year_total - ELSE NULL END -ORDER BY - t_s_secyear.customer_id, - t_s_secyear.customer_first_name, - t_s_secyear.customer_last_name, - t_s_secyear.customer_preferred_cust_flag, - t_s_secyear.customer_birth_country, - t_s_secyear.customer_login, - t_s_secyear.customer_email_address -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q40.sql b/src/test/resources/tpcds/queries/q40.sql deleted file mode 100644 index 66d8b73ac..000000000 --- a/src/test/resources/tpcds/queries/q40.sql +++ /dev/null @@ -1,25 +0,0 @@ -SELECT - w_state, - i_item_id, - sum(CASE WHEN (cast(d_date AS DATE) < cast('2000-03-11' AS DATE)) - THEN cs_sales_price - coalesce(cr_refunded_cash, 0) - ELSE 0 END) AS sales_before, - sum(CASE WHEN (cast(d_date AS DATE) >= cast('2000-03-11' AS DATE)) - THEN cs_sales_price - coalesce(cr_refunded_cash, 0) - ELSE 0 END) AS sales_after -FROM - catalog_sales - LEFT OUTER JOIN catalog_returns ON - (cs_order_number = cr_order_number - AND cs_item_sk = cr_item_sk) - , warehouse, item, date_dim -WHERE - i_current_price BETWEEN 0.99 AND 1.49 - AND i_item_sk = cs_item_sk - AND cs_warehouse_sk = w_warehouse_sk - AND cs_sold_date_sk = d_date_sk - AND d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) - AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days) -GROUP BY w_state, i_item_id -ORDER BY w_state, i_item_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q41.sql b/src/test/resources/tpcds/queries/q41.sql deleted file mode 100644 index 25e317e0e..000000000 --- a/src/test/resources/tpcds/queries/q41.sql +++ /dev/null @@ -1,49 +0,0 @@ -SELECT DISTINCT (i_product_name) -FROM item i1 -WHERE i_manufact_id BETWEEN 738 AND 738 + 40 - AND (SELECT count(*) AS item_cnt -FROM item -WHERE (i_manufact = i1.i_manufact AND - ((i_category = 'Women' AND - (i_color = 'powder' OR i_color = 'khaki') AND - (i_units = 'Ounce' OR i_units = 'Oz') AND - (i_size = 'medium' OR i_size = 'extra large') - ) OR - (i_category = 'Women' AND - (i_color = 'brown' OR i_color = 'honeydew') AND - (i_units = 'Bunch' OR i_units = 'Ton') AND - (i_size = 'N/A' OR i_size = 'small') - ) OR - (i_category = 'Men' AND - (i_color = 'floral' OR i_color = 'deep') AND - (i_units = 'N/A' OR i_units = 'Dozen') AND - (i_size = 'petite' OR i_size = 'large') - ) OR - (i_category = 'Men' AND - (i_color = 'light' OR i_color = 'cornflower') AND - (i_units = 'Box' OR i_units = 'Pound') AND - (i_size = 'medium' OR i_size = 'extra large') - ))) OR - (i_manufact = i1.i_manufact AND - ((i_category = 'Women' AND - (i_color = 'midnight' OR i_color = 'snow') AND - (i_units = 'Pallet' OR i_units = 'Gross') AND - (i_size = 'medium' OR i_size = 'extra large') - ) OR - (i_category = 'Women' AND - (i_color = 'cyan' OR i_color = 'papaya') AND - (i_units = 'Cup' OR i_units = 'Dram') AND - (i_size = 'N/A' OR i_size = 'small') - ) OR - (i_category = 'Men' AND - (i_color = 'orange' OR i_color = 'frosted') AND - (i_units = 'Each' OR i_units = 'Tbl') AND - (i_size = 'petite' OR i_size = 'large') - ) OR - (i_category = 'Men' AND - (i_color = 'forest' OR i_color = 'ghost') AND - (i_units = 'Lb' OR i_units = 'Bundle') AND - (i_size = 'medium' OR i_size = 'extra large') - )))) > 0 -ORDER BY i_product_name -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q42.sql b/src/test/resources/tpcds/queries/q42.sql deleted file mode 100644 index 4d2e71760..000000000 --- a/src/test/resources/tpcds/queries/q42.sql +++ /dev/null @@ -1,18 +0,0 @@ -SELECT - dt.d_year, - item.i_category_id, - item.i_category, - sum(ss_ext_sales_price) -FROM date_dim dt, store_sales, item -WHERE dt.d_date_sk = store_sales.ss_sold_date_sk - AND store_sales.ss_item_sk = item.i_item_sk - AND item.i_manager_id = 1 - AND dt.d_moy = 11 - AND dt.d_year = 2000 -GROUP BY dt.d_year - , item.i_category_id - , item.i_category -ORDER BY sum(ss_ext_sales_price) DESC, dt.d_year - , item.i_category_id - , item.i_category -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q43.sql b/src/test/resources/tpcds/queries/q43.sql deleted file mode 100644 index 45411772c..000000000 --- a/src/test/resources/tpcds/queries/q43.sql +++ /dev/null @@ -1,33 +0,0 @@ -SELECT - s_store_name, - s_store_id, - sum(CASE WHEN (d_day_name = 'Sunday') - THEN ss_sales_price - ELSE NULL END) sun_sales, - sum(CASE WHEN (d_day_name = 'Monday') - THEN ss_sales_price - ELSE NULL END) mon_sales, - sum(CASE WHEN (d_day_name = 'Tuesday') - THEN ss_sales_price - ELSE NULL END) tue_sales, - sum(CASE WHEN (d_day_name = 'Wednesday') - THEN ss_sales_price - ELSE NULL END) wed_sales, - sum(CASE WHEN (d_day_name = 'Thursday') - THEN ss_sales_price - ELSE NULL END) thu_sales, - sum(CASE WHEN (d_day_name = 'Friday') - THEN ss_sales_price - ELSE NULL END) fri_sales, - sum(CASE WHEN (d_day_name = 'Saturday') - THEN ss_sales_price - ELSE NULL END) sat_sales -FROM date_dim, store_sales, store -WHERE d_date_sk = ss_sold_date_sk AND - s_store_sk = ss_store_sk AND - s_gmt_offset = -5 AND - d_year = 2000 -GROUP BY s_store_name, s_store_id -ORDER BY s_store_name, s_store_id, sun_sales, mon_sales, tue_sales, wed_sales, - thu_sales, fri_sales, sat_sales -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q44.sql b/src/test/resources/tpcds/queries/q44.sql deleted file mode 100644 index 379e60478..000000000 --- a/src/test/resources/tpcds/queries/q44.sql +++ /dev/null @@ -1,46 +0,0 @@ -SELECT - asceding.rnk, - i1.i_product_name best_performing, - i2.i_product_name worst_performing -FROM (SELECT * -FROM (SELECT - item_sk, - rank() - OVER ( - ORDER BY rank_col ASC) rnk -FROM (SELECT - ss_item_sk item_sk, - avg(ss_net_profit) rank_col -FROM store_sales ss1 -WHERE ss_store_sk = 4 -GROUP BY ss_item_sk -HAVING avg(ss_net_profit) > 0.9 * (SELECT avg(ss_net_profit) rank_col -FROM store_sales -WHERE ss_store_sk = 4 - AND ss_addr_sk IS NULL -GROUP BY ss_store_sk)) V1) V11 -WHERE rnk < 11) asceding, - (SELECT * - FROM (SELECT - item_sk, - rank() - OVER ( - ORDER BY rank_col DESC) rnk - FROM (SELECT - ss_item_sk item_sk, - avg(ss_net_profit) rank_col - FROM store_sales ss1 - WHERE ss_store_sk = 4 - GROUP BY ss_item_sk - HAVING avg(ss_net_profit) > 0.9 * (SELECT avg(ss_net_profit) rank_col - FROM store_sales - WHERE ss_store_sk = 4 - AND ss_addr_sk IS NULL - GROUP BY ss_store_sk)) V2) V21 - WHERE rnk < 11) descending, - item i1, item i2 -WHERE asceding.rnk = descending.rnk - AND i1.i_item_sk = asceding.item_sk - AND i2.i_item_sk = descending.item_sk -ORDER BY asceding.rnk -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q45.sql b/src/test/resources/tpcds/queries/q45.sql deleted file mode 100644 index 907438f19..000000000 --- a/src/test/resources/tpcds/queries/q45.sql +++ /dev/null @@ -1,21 +0,0 @@ -SELECT - ca_zip, - ca_city, - sum(ws_sales_price) -FROM web_sales, customer, customer_address, date_dim, item -WHERE ws_bill_customer_sk = c_customer_sk - AND c_current_addr_sk = ca_address_sk - AND ws_item_sk = i_item_sk - AND (substr(ca_zip, 1, 5) IN - ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') - OR - i_item_id IN (SELECT i_item_id - FROM item - WHERE i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) - ) -) - AND ws_sold_date_sk = d_date_sk - AND d_qoy = 2 AND d_year = 2001 -GROUP BY ca_zip, ca_city -ORDER BY ca_zip, ca_city -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q46.sql b/src/test/resources/tpcds/queries/q46.sql deleted file mode 100644 index 0911677df..000000000 --- a/src/test/resources/tpcds/queries/q46.sql +++ /dev/null @@ -1,32 +0,0 @@ -SELECT - c_last_name, - c_first_name, - ca_city, - bought_city, - ss_ticket_number, - amt, - profit -FROM - (SELECT - ss_ticket_number, - ss_customer_sk, - ca_city bought_city, - sum(ss_coupon_amt) amt, - sum(ss_net_profit) profit - FROM store_sales, date_dim, store, household_demographics, customer_address - WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk - AND store_sales.ss_store_sk = store.s_store_sk - AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk - AND store_sales.ss_addr_sk = customer_address.ca_address_sk - AND (household_demographics.hd_dep_count = 4 OR - household_demographics.hd_vehicle_count = 3) - AND date_dim.d_dow IN (6, 0) - AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) - AND store.s_city IN ('Fairview', 'Midway', 'Fairview', 'Fairview', 'Fairview') - GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city) dn, customer, - customer_address current_addr -WHERE ss_customer_sk = c_customer_sk - AND customer.c_current_addr_sk = current_addr.ca_address_sk - AND current_addr.ca_city <> bought_city -ORDER BY c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q47.sql b/src/test/resources/tpcds/queries/q47.sql deleted file mode 100644 index cfc37a4ce..000000000 --- a/src/test/resources/tpcds/queries/q47.sql +++ /dev/null @@ -1,63 +0,0 @@ -WITH v1 AS ( - SELECT - i_category, - i_brand, - s_store_name, - s_company_name, - d_year, - d_moy, - sum(ss_sales_price) sum_sales, - avg(sum(ss_sales_price)) - OVER - (PARTITION BY i_category, i_brand, - s_store_name, s_company_name, d_year) - avg_monthly_sales, - rank() - OVER - (PARTITION BY i_category, i_brand, - s_store_name, s_company_name - ORDER BY d_year, d_moy) rn - FROM item, store_sales, date_dim, store - WHERE ss_item_sk = i_item_sk AND - ss_sold_date_sk = d_date_sk AND - ss_store_sk = s_store_sk AND - ( - d_year = 1999 OR - (d_year = 1999 - 1 AND d_moy = 12) OR - (d_year = 1999 + 1 AND d_moy = 1) - ) - GROUP BY i_category, i_brand, - s_store_name, s_company_name, - d_year, d_moy), - v2 AS ( - SELECT - v1.i_category, - v1.i_brand, - v1.s_store_name, - v1.s_company_name, - v1.d_year, - v1.d_moy, - v1.avg_monthly_sales, - v1.sum_sales, - v1_lag.sum_sales psum, - v1_lead.sum_sales nsum - FROM v1, v1 v1_lag, v1 v1_lead - WHERE v1.i_category = v1_lag.i_category AND - v1.i_category = v1_lead.i_category AND - v1.i_brand = v1_lag.i_brand AND - v1.i_brand = v1_lead.i_brand AND - v1.s_store_name = v1_lag.s_store_name AND - v1.s_store_name = v1_lead.s_store_name AND - v1.s_company_name = v1_lag.s_company_name AND - v1.s_company_name = v1_lead.s_company_name AND - v1.rn = v1_lag.rn + 1 AND - v1.rn = v1_lead.rn - 1) -SELECT * -FROM v2 -WHERE d_year = 1999 AND - avg_monthly_sales > 0 AND - CASE WHEN avg_monthly_sales > 0 - THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales - ELSE NULL END > 0.1 -ORDER BY sum_sales - avg_monthly_sales, 3 -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q48.sql b/src/test/resources/tpcds/queries/q48.sql deleted file mode 100644 index fdb9f38e2..000000000 --- a/src/test/resources/tpcds/queries/q48.sql +++ /dev/null @@ -1,63 +0,0 @@ -SELECT sum(ss_quantity) -FROM store_sales, store, customer_demographics, customer_address, date_dim -WHERE s_store_sk = ss_store_sk - AND ss_sold_date_sk = d_date_sk AND d_year = 2001 - AND - ( - ( - cd_demo_sk = ss_cdemo_sk - AND - cd_marital_status = 'M' - AND - cd_education_status = '4 yr Degree' - AND - ss_sales_price BETWEEN 100.00 AND 150.00 - ) - OR - ( - cd_demo_sk = ss_cdemo_sk - AND - cd_marital_status = 'D' - AND - cd_education_status = '2 yr Degree' - AND - ss_sales_price BETWEEN 50.00 AND 100.00 - ) - OR - ( - cd_demo_sk = ss_cdemo_sk - AND - cd_marital_status = 'S' - AND - cd_education_status = 'College' - AND - ss_sales_price BETWEEN 150.00 AND 200.00 - ) - ) - AND - ( - ( - ss_addr_sk = ca_address_sk - AND - ca_country = 'United States' - AND - ca_state IN ('CO', 'OH', 'TX') - AND ss_net_profit BETWEEN 0 AND 2000 - ) - OR - (ss_addr_sk = ca_address_sk - AND - ca_country = 'United States' - AND - ca_state IN ('OR', 'MN', 'KY') - AND ss_net_profit BETWEEN 150 AND 3000 - ) - OR - (ss_addr_sk = ca_address_sk - AND - ca_country = 'United States' - AND - ca_state IN ('VA', 'CA', 'MS') - AND ss_net_profit BETWEEN 50 AND 25000 - ) - ) diff --git a/src/test/resources/tpcds/queries/q49.sql b/src/test/resources/tpcds/queries/q49.sql deleted file mode 100644 index 9568d8b92..000000000 --- a/src/test/resources/tpcds/queries/q49.sql +++ /dev/null @@ -1,126 +0,0 @@ -SELECT - 'web' AS channel, - web.item, - web.return_ratio, - web.return_rank, - web.currency_rank -FROM ( - SELECT - item, - return_ratio, - currency_ratio, - rank() - OVER ( - ORDER BY return_ratio) AS return_rank, - rank() - OVER ( - ORDER BY currency_ratio) AS currency_rank - FROM - (SELECT - ws.ws_item_sk AS item, - (cast(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15, 4)) / - cast(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, - (cast(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15, 4)) / - cast(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio - FROM - web_sales ws LEFT OUTER JOIN web_returns wr - ON (ws.ws_order_number = wr.wr_order_number AND - ws.ws_item_sk = wr.wr_item_sk) - , date_dim - WHERE - wr.wr_return_amt > 10000 - AND ws.ws_net_profit > 1 - AND ws.ws_net_paid > 0 - AND ws.ws_quantity > 0 - AND ws_sold_date_sk = d_date_sk - AND d_year = 2001 - AND d_moy = 12 - GROUP BY ws.ws_item_sk - ) in_web - ) web -WHERE (web.return_rank <= 10 OR web.currency_rank <= 10) -UNION -SELECT - 'catalog' AS channel, - catalog.item, - catalog.return_ratio, - catalog.return_rank, - catalog.currency_rank -FROM ( - SELECT - item, - return_ratio, - currency_ratio, - rank() - OVER ( - ORDER BY return_ratio) AS return_rank, - rank() - OVER ( - ORDER BY currency_ratio) AS currency_rank - FROM - (SELECT - cs.cs_item_sk AS item, - (cast(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15, 4)) / - cast(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, - (cast(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15, 4)) / - cast(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio - FROM - catalog_sales cs LEFT OUTER JOIN catalog_returns cr - ON (cs.cs_order_number = cr.cr_order_number AND - cs.cs_item_sk = cr.cr_item_sk) - , date_dim - WHERE - cr.cr_return_amount > 10000 - AND cs.cs_net_profit > 1 - AND cs.cs_net_paid > 0 - AND cs.cs_quantity > 0 - AND cs_sold_date_sk = d_date_sk - AND d_year = 2001 - AND d_moy = 12 - GROUP BY cs.cs_item_sk - ) in_cat - ) catalog -WHERE (catalog.return_rank <= 10 OR catalog.currency_rank <= 10) -UNION -SELECT - 'store' AS channel, - store.item, - store.return_ratio, - store.return_rank, - store.currency_rank -FROM ( - SELECT - item, - return_ratio, - currency_ratio, - rank() - OVER ( - ORDER BY return_ratio) AS return_rank, - rank() - OVER ( - ORDER BY currency_ratio) AS currency_rank - FROM - (SELECT - sts.ss_item_sk AS item, - (cast(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15, 4)) / - cast(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, - (cast(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15, 4)) / - cast(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio - FROM - store_sales sts LEFT OUTER JOIN store_returns sr - ON (sts.ss_ticket_number = sr.sr_ticket_number AND sts.ss_item_sk = sr.sr_item_sk) - , date_dim - WHERE - sr.sr_return_amt > 10000 - AND sts.ss_net_profit > 1 - AND sts.ss_net_paid > 0 - AND sts.ss_quantity > 0 - AND ss_sold_date_sk = d_date_sk - AND d_year = 2001 - AND d_moy = 12 - GROUP BY sts.ss_item_sk - ) in_store - ) store -WHERE (store.return_rank <= 10 OR store.currency_rank <= 10) -ORDER BY 1, 4, 5 -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q5.sql b/src/test/resources/tpcds/queries/q5.sql deleted file mode 100644 index b87cf3a44..000000000 --- a/src/test/resources/tpcds/queries/q5.sql +++ /dev/null @@ -1,131 +0,0 @@ -WITH ssr AS -( SELECT - s_store_id, - sum(sales_price) AS sales, - sum(profit) AS profit, - sum(return_amt) AS RETURNS, - sum(net_loss) AS profit_loss - FROM - (SELECT - ss_store_sk AS store_sk, - ss_sold_date_sk AS date_sk, - ss_ext_sales_price AS sales_price, - ss_net_profit AS profit, - cast(0 AS DECIMAL(7, 2)) AS return_amt, - cast(0 AS DECIMAL(7, 2)) AS net_loss - FROM store_sales - UNION ALL - SELECT - sr_store_sk AS store_sk, - sr_returned_date_sk AS date_sk, - cast(0 AS DECIMAL(7, 2)) AS sales_price, - cast(0 AS DECIMAL(7, 2)) AS profit, - sr_return_amt AS return_amt, - sr_net_loss AS net_loss - FROM store_returns) - salesreturns, date_dim, store - WHERE date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-23' AS DATE) - AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) - AND store_sk = s_store_sk - GROUP BY s_store_id), - csr AS - ( SELECT - cp_catalog_page_id, - sum(sales_price) AS sales, - sum(profit) AS profit, - sum(return_amt) AS RETURNS, - sum(net_loss) AS profit_loss - FROM - (SELECT - cs_catalog_page_sk AS page_sk, - cs_sold_date_sk AS date_sk, - cs_ext_sales_price AS sales_price, - cs_net_profit AS profit, - cast(0 AS DECIMAL(7, 2)) AS return_amt, - cast(0 AS DECIMAL(7, 2)) AS net_loss - FROM catalog_sales - UNION ALL - SELECT - cr_catalog_page_sk AS page_sk, - cr_returned_date_sk AS date_sk, - cast(0 AS DECIMAL(7, 2)) AS sales_price, - cast(0 AS DECIMAL(7, 2)) AS profit, - cr_return_amount AS return_amt, - cr_net_loss AS net_loss - FROM catalog_returns - ) salesreturns, date_dim, catalog_page - WHERE date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-23' AS DATE) - AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) - AND page_sk = cp_catalog_page_sk - GROUP BY cp_catalog_page_id) - , - wsr AS - ( SELECT - web_site_id, - sum(sales_price) AS sales, - sum(profit) AS profit, - sum(return_amt) AS RETURNS, - sum(net_loss) AS profit_loss - FROM - (SELECT - ws_web_site_sk AS wsr_web_site_sk, - ws_sold_date_sk AS date_sk, - ws_ext_sales_price AS sales_price, - ws_net_profit AS profit, - cast(0 AS DECIMAL(7, 2)) AS return_amt, - cast(0 AS DECIMAL(7, 2)) AS net_loss - FROM web_sales - UNION ALL - SELECT - ws_web_site_sk AS wsr_web_site_sk, - wr_returned_date_sk AS date_sk, - cast(0 AS DECIMAL(7, 2)) AS sales_price, - cast(0 AS DECIMAL(7, 2)) AS profit, - wr_return_amt AS return_amt, - wr_net_loss AS net_loss - FROM web_returns - LEFT OUTER JOIN web_sales ON - (wr_item_sk = ws_item_sk - AND wr_order_number = ws_order_number) - ) salesreturns, date_dim, web_site - WHERE date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-23' AS DATE) - AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) - AND wsr_web_site_sk = web_site_sk - GROUP BY web_site_id) -SELECT - channel, - id, - sum(sales) AS sales, - sum(returns) AS returns, - sum(profit) AS profit -FROM - (SELECT - 'store channel' AS channel, - concat('store', s_store_id) AS id, - sales, - returns, - (profit - profit_loss) AS profit - FROM ssr - UNION ALL - SELECT - 'catalog channel' AS channel, - concat('catalog_page', cp_catalog_page_id) AS id, - sales, - returns, - (profit - profit_loss) AS profit - FROM csr - UNION ALL - SELECT - 'web channel' AS channel, - concat('web_site', web_site_id) AS id, - sales, - returns, - (profit - profit_loss) AS profit - FROM wsr - ) x -GROUP BY ROLLUP (channel, id) -ORDER BY channel, id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q50.sql b/src/test/resources/tpcds/queries/q50.sql deleted file mode 100644 index f1d4b1544..000000000 --- a/src/test/resources/tpcds/queries/q50.sql +++ /dev/null @@ -1,47 +0,0 @@ -SELECT - s_store_name, - s_company_id, - s_street_number, - s_street_name, - s_street_type, - s_suite_number, - s_city, - s_county, - s_state, - s_zip, - sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk <= 30) - THEN 1 - ELSE 0 END) AS `30 days `, - sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 30) AND - (sr_returned_date_sk - ss_sold_date_sk <= 60) - THEN 1 - ELSE 0 END) AS `31 - 60 days `, - sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 60) AND - (sr_returned_date_sk - ss_sold_date_sk <= 90) - THEN 1 - ELSE 0 END) AS `61 - 90 days `, - sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 90) AND - (sr_returned_date_sk - ss_sold_date_sk <= 120) - THEN 1 - ELSE 0 END) AS `91 - 120 days `, - sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 120) - THEN 1 - ELSE 0 END) AS `>120 days ` -FROM - store_sales, store_returns, store, date_dim d1, date_dim d2 -WHERE - d2.d_year = 2001 - AND d2.d_moy = 8 - AND ss_ticket_number = sr_ticket_number - AND ss_item_sk = sr_item_sk - AND ss_sold_date_sk = d1.d_date_sk - AND sr_returned_date_sk = d2.d_date_sk - AND ss_customer_sk = sr_customer_sk - AND ss_store_sk = s_store_sk -GROUP BY - s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, - s_suite_number, s_city, s_county, s_state, s_zip -ORDER BY - s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, - s_suite_number, s_city, s_county, s_state, s_zip -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q51.sql b/src/test/resources/tpcds/queries/q51.sql deleted file mode 100644 index 62b003eb6..000000000 --- a/src/test/resources/tpcds/queries/q51.sql +++ /dev/null @@ -1,55 +0,0 @@ -WITH web_v1 AS ( - SELECT - ws_item_sk item_sk, - d_date, - sum(sum(ws_sales_price)) - OVER (PARTITION BY ws_item_sk - ORDER BY d_date - ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) cume_sales - FROM web_sales, date_dim - WHERE ws_sold_date_sk = d_date_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11 - AND ws_item_sk IS NOT NULL - GROUP BY ws_item_sk, d_date), - store_v1 AS ( - SELECT - ss_item_sk item_sk, - d_date, - sum(sum(ss_sales_price)) - OVER (PARTITION BY ss_item_sk - ORDER BY d_date - ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) cume_sales - FROM store_sales, date_dim - WHERE ss_sold_date_sk = d_date_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11 - AND ss_item_sk IS NOT NULL - GROUP BY ss_item_sk, d_date) -SELECT * -FROM (SELECT - item_sk, - d_date, - web_sales, - store_sales, - max(web_sales) - OVER (PARTITION BY item_sk - ORDER BY d_date - ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) web_cumulative, - max(store_sales) - OVER (PARTITION BY item_sk - ORDER BY d_date - ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) store_cumulative -FROM (SELECT - CASE WHEN web.item_sk IS NOT NULL - THEN web.item_sk - ELSE store.item_sk END item_sk, - CASE WHEN web.d_date IS NOT NULL - THEN web.d_date - ELSE store.d_date END d_date, - web.cume_sales web_sales, - store.cume_sales store_sales -FROM web_v1 web FULL OUTER JOIN store_v1 store ON (web.item_sk = store.item_sk - AND web.d_date = store.d_date) - ) x) y -WHERE web_cumulative > store_cumulative -ORDER BY item_sk, d_date -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q52.sql b/src/test/resources/tpcds/queries/q52.sql deleted file mode 100644 index 467d1ae05..000000000 --- a/src/test/resources/tpcds/queries/q52.sql +++ /dev/null @@ -1,14 +0,0 @@ -SELECT - dt.d_year, - item.i_brand_id brand_id, - item.i_brand brand, - sum(ss_ext_sales_price) ext_price -FROM date_dim dt, store_sales, item -WHERE dt.d_date_sk = store_sales.ss_sold_date_sk - AND store_sales.ss_item_sk = item.i_item_sk - AND item.i_manager_id = 1 - AND dt.d_moy = 11 - AND dt.d_year = 2000 -GROUP BY dt.d_year, item.i_brand, item.i_brand_id -ORDER BY dt.d_year, ext_price DESC, brand_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q53.sql b/src/test/resources/tpcds/queries/q53.sql deleted file mode 100644 index b42c68dcf..000000000 --- a/src/test/resources/tpcds/queries/q53.sql +++ /dev/null @@ -1,30 +0,0 @@ -SELECT * -FROM - (SELECT - i_manufact_id, - sum(ss_sales_price) sum_sales, - avg(sum(ss_sales_price)) - OVER (PARTITION BY i_manufact_id) avg_quarterly_sales - FROM item, store_sales, date_dim, store - WHERE ss_item_sk = i_item_sk AND - ss_sold_date_sk = d_date_sk AND - ss_store_sk = s_store_sk AND - d_month_seq IN (1200, 1200 + 1, 1200 + 2, 1200 + 3, 1200 + 4, 1200 + 5, 1200 + 6, - 1200 + 7, 1200 + 8, 1200 + 9, 1200 + 10, 1200 + 11) AND - ((i_category IN ('Books', 'Children', 'Electronics') AND - i_class IN ('personal', 'portable', 'reference', 'self-help') AND - i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', - 'exportiunivamalg #9', 'scholaramalgamalg #9')) - OR - (i_category IN ('Women', 'Music', 'Men') AND - i_class IN ('accessories', 'classical', 'fragrances', 'pants') AND - i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', - 'importoamalg #1'))) - GROUP BY i_manufact_id, d_qoy) tmp1 -WHERE CASE WHEN avg_quarterly_sales > 0 - THEN abs(sum_sales - avg_quarterly_sales) / avg_quarterly_sales - ELSE NULL END > 0.1 -ORDER BY avg_quarterly_sales, - sum_sales, - i_manufact_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q54.sql b/src/test/resources/tpcds/queries/q54.sql deleted file mode 100644 index 897237fb6..000000000 --- a/src/test/resources/tpcds/queries/q54.sql +++ /dev/null @@ -1,61 +0,0 @@ -WITH my_customers AS ( - SELECT DISTINCT - c_customer_sk, - c_current_addr_sk - FROM - (SELECT - cs_sold_date_sk sold_date_sk, - cs_bill_customer_sk customer_sk, - cs_item_sk item_sk - FROM catalog_sales - UNION ALL - SELECT - ws_sold_date_sk sold_date_sk, - ws_bill_customer_sk customer_sk, - ws_item_sk item_sk - FROM web_sales - ) cs_or_ws_sales, - item, - date_dim, - customer - WHERE sold_date_sk = d_date_sk - AND item_sk = i_item_sk - AND i_category = 'Women' - AND i_class = 'maternity' - AND c_customer_sk = cs_or_ws_sales.customer_sk - AND d_moy = 12 - AND d_year = 1998 -) - , my_revenue AS ( - SELECT - c_customer_sk, - sum(ss_ext_sales_price) AS revenue - FROM my_customers, - store_sales, - customer_address, - store, - date_dim - WHERE c_current_addr_sk = ca_address_sk - AND ca_county = s_county - AND ca_state = s_state - AND ss_sold_date_sk = d_date_sk - AND c_customer_sk = ss_customer_sk - AND d_month_seq BETWEEN (SELECT DISTINCT d_month_seq + 1 - FROM date_dim - WHERE d_year = 1998 AND d_moy = 12) - AND (SELECT DISTINCT d_month_seq + 3 - FROM date_dim - WHERE d_year = 1998 AND d_moy = 12) - GROUP BY c_customer_sk -) - , segments AS -(SELECT cast((revenue / 50) AS INT) AS segment - FROM my_revenue) -SELECT - segment, - count(*) AS num_customers, - segment * 50 AS segment_base -FROM segments -GROUP BY segment -ORDER BY segment, num_customers -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q55.sql b/src/test/resources/tpcds/queries/q55.sql deleted file mode 100644 index bc5d888c9..000000000 --- a/src/test/resources/tpcds/queries/q55.sql +++ /dev/null @@ -1,13 +0,0 @@ -SELECT - i_brand_id brand_id, - i_brand brand, - sum(ss_ext_sales_price) ext_price -FROM date_dim, store_sales, item -WHERE d_date_sk = ss_sold_date_sk - AND ss_item_sk = i_item_sk - AND i_manager_id = 28 - AND d_moy = 11 - AND d_year = 1999 -GROUP BY i_brand, i_brand_id -ORDER BY ext_price DESC, brand_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q56.sql b/src/test/resources/tpcds/queries/q56.sql deleted file mode 100644 index 2fa1738dc..000000000 --- a/src/test/resources/tpcds/queries/q56.sql +++ /dev/null @@ -1,65 +0,0 @@ -WITH ss AS ( - SELECT - i_item_id, - sum(ss_ext_sales_price) total_sales - FROM - store_sales, date_dim, customer_address, item - WHERE - i_item_id IN (SELECT i_item_id - FROM item - WHERE i_color IN ('slate', 'blanched', 'burnished')) - AND ss_item_sk = i_item_sk - AND ss_sold_date_sk = d_date_sk - AND d_year = 2001 - AND d_moy = 2 - AND ss_addr_sk = ca_address_sk - AND ca_gmt_offset = -5 - GROUP BY i_item_id), - cs AS ( - SELECT - i_item_id, - sum(cs_ext_sales_price) total_sales - FROM - catalog_sales, date_dim, customer_address, item - WHERE - i_item_id IN (SELECT i_item_id - FROM item - WHERE i_color IN ('slate', 'blanched', 'burnished')) - AND cs_item_sk = i_item_sk - AND cs_sold_date_sk = d_date_sk - AND d_year = 2001 - AND d_moy = 2 - AND cs_bill_addr_sk = ca_address_sk - AND ca_gmt_offset = -5 - GROUP BY i_item_id), - ws AS ( - SELECT - i_item_id, - sum(ws_ext_sales_price) total_sales - FROM - web_sales, date_dim, customer_address, item - WHERE - i_item_id IN (SELECT i_item_id - FROM item - WHERE i_color IN ('slate', 'blanched', 'burnished')) - AND ws_item_sk = i_item_sk - AND ws_sold_date_sk = d_date_sk - AND d_year = 2001 - AND d_moy = 2 - AND ws_bill_addr_sk = ca_address_sk - AND ca_gmt_offset = -5 - GROUP BY i_item_id) -SELECT - i_item_id, - sum(total_sales) total_sales -FROM (SELECT * - FROM ss - UNION ALL - SELECT * - FROM cs - UNION ALL - SELECT * - FROM ws) tmp1 -GROUP BY i_item_id -ORDER BY total_sales -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q57.sql b/src/test/resources/tpcds/queries/q57.sql deleted file mode 100644 index cf70d4b90..000000000 --- a/src/test/resources/tpcds/queries/q57.sql +++ /dev/null @@ -1,56 +0,0 @@ -WITH v1 AS ( - SELECT - i_category, - i_brand, - cc_name, - d_year, - d_moy, - sum(cs_sales_price) sum_sales, - avg(sum(cs_sales_price)) - OVER - (PARTITION BY i_category, i_brand, cc_name, d_year) - avg_monthly_sales, - rank() - OVER - (PARTITION BY i_category, i_brand, cc_name - ORDER BY d_year, d_moy) rn - FROM item, catalog_sales, date_dim, call_center - WHERE cs_item_sk = i_item_sk AND - cs_sold_date_sk = d_date_sk AND - cc_call_center_sk = cs_call_center_sk AND - ( - d_year = 1999 OR - (d_year = 1999 - 1 AND d_moy = 12) OR - (d_year = 1999 + 1 AND d_moy = 1) - ) - GROUP BY i_category, i_brand, - cc_name, d_year, d_moy), - v2 AS ( - SELECT - v1.i_category, - v1.i_brand, - v1.cc_name, - v1.d_year, - v1.d_moy, - v1.avg_monthly_sales, - v1.sum_sales, - v1_lag.sum_sales psum, - v1_lead.sum_sales nsum - FROM v1, v1 v1_lag, v1 v1_lead - WHERE v1.i_category = v1_lag.i_category AND - v1.i_category = v1_lead.i_category AND - v1.i_brand = v1_lag.i_brand AND - v1.i_brand = v1_lead.i_brand AND - v1.cc_name = v1_lag.cc_name AND - v1.cc_name = v1_lead.cc_name AND - v1.rn = v1_lag.rn + 1 AND - v1.rn = v1_lead.rn - 1) -SELECT * -FROM v2 -WHERE d_year = 1999 AND - avg_monthly_sales > 0 AND - CASE WHEN avg_monthly_sales > 0 - THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales - ELSE NULL END > 0.1 -ORDER BY sum_sales - avg_monthly_sales, 3 -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q58.sql b/src/test/resources/tpcds/queries/q58.sql deleted file mode 100644 index 5f63f33dc..000000000 --- a/src/test/resources/tpcds/queries/q58.sql +++ /dev/null @@ -1,59 +0,0 @@ -WITH ss_items AS -(SELECT - i_item_id item_id, - sum(ss_ext_sales_price) ss_item_rev - FROM store_sales, item, date_dim - WHERE ss_item_sk = i_item_sk - AND d_date IN (SELECT d_date - FROM date_dim - WHERE d_week_seq = (SELECT d_week_seq - FROM date_dim - WHERE d_date = '2000-01-03')) - AND ss_sold_date_sk = d_date_sk - GROUP BY i_item_id), - cs_items AS - (SELECT - i_item_id item_id, - sum(cs_ext_sales_price) cs_item_rev - FROM catalog_sales, item, date_dim - WHERE cs_item_sk = i_item_sk - AND d_date IN (SELECT d_date - FROM date_dim - WHERE d_week_seq = (SELECT d_week_seq - FROM date_dim - WHERE d_date = '2000-01-03')) - AND cs_sold_date_sk = d_date_sk - GROUP BY i_item_id), - ws_items AS - (SELECT - i_item_id item_id, - sum(ws_ext_sales_price) ws_item_rev - FROM web_sales, item, date_dim - WHERE ws_item_sk = i_item_sk - AND d_date IN (SELECT d_date - FROM date_dim - WHERE d_week_seq = (SELECT d_week_seq - FROM date_dim - WHERE d_date = '2000-01-03')) - AND ws_sold_date_sk = d_date_sk - GROUP BY i_item_id) -SELECT - ss_items.item_id, - ss_item_rev, - ss_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 ss_dev, - cs_item_rev, - cs_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 cs_dev, - ws_item_rev, - ws_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 ws_dev, - (ss_item_rev + cs_item_rev + ws_item_rev) / 3 average -FROM ss_items, cs_items, ws_items -WHERE ss_items.item_id = cs_items.item_id - AND ss_items.item_id = ws_items.item_id - AND ss_item_rev BETWEEN 0.9 * cs_item_rev AND 1.1 * cs_item_rev - AND ss_item_rev BETWEEN 0.9 * ws_item_rev AND 1.1 * ws_item_rev - AND cs_item_rev BETWEEN 0.9 * ss_item_rev AND 1.1 * ss_item_rev - AND cs_item_rev BETWEEN 0.9 * ws_item_rev AND 1.1 * ws_item_rev - AND ws_item_rev BETWEEN 0.9 * ss_item_rev AND 1.1 * ss_item_rev - AND ws_item_rev BETWEEN 0.9 * cs_item_rev AND 1.1 * cs_item_rev -ORDER BY item_id, ss_item_rev -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q59.sql b/src/test/resources/tpcds/queries/q59.sql deleted file mode 100644 index 3cef20276..000000000 --- a/src/test/resources/tpcds/queries/q59.sql +++ /dev/null @@ -1,75 +0,0 @@ -WITH wss AS -(SELECT - d_week_seq, - ss_store_sk, - sum(CASE WHEN (d_day_name = 'Sunday') - THEN ss_sales_price - ELSE NULL END) sun_sales, - sum(CASE WHEN (d_day_name = 'Monday') - THEN ss_sales_price - ELSE NULL END) mon_sales, - sum(CASE WHEN (d_day_name = 'Tuesday') - THEN ss_sales_price - ELSE NULL END) tue_sales, - sum(CASE WHEN (d_day_name = 'Wednesday') - THEN ss_sales_price - ELSE NULL END) wed_sales, - sum(CASE WHEN (d_day_name = 'Thursday') - THEN ss_sales_price - ELSE NULL END) thu_sales, - sum(CASE WHEN (d_day_name = 'Friday') - THEN ss_sales_price - ELSE NULL END) fri_sales, - sum(CASE WHEN (d_day_name = 'Saturday') - THEN ss_sales_price - ELSE NULL END) sat_sales - FROM store_sales, date_dim - WHERE d_date_sk = ss_sold_date_sk - GROUP BY d_week_seq, ss_store_sk -) -SELECT - s_store_name1, - s_store_id1, - d_week_seq1, - sun_sales1 / sun_sales2, - mon_sales1 / mon_sales2, - tue_sales1 / tue_sales2, - wed_sales1 / wed_sales2, - thu_sales1 / thu_sales2, - fri_sales1 / fri_sales2, - sat_sales1 / sat_sales2 -FROM - (SELECT - s_store_name s_store_name1, - wss.d_week_seq d_week_seq1, - s_store_id s_store_id1, - sun_sales sun_sales1, - mon_sales mon_sales1, - tue_sales tue_sales1, - wed_sales wed_sales1, - thu_sales thu_sales1, - fri_sales fri_sales1, - sat_sales sat_sales1 - FROM wss, store, date_dim d - WHERE d.d_week_seq = wss.d_week_seq AND - ss_store_sk = s_store_sk AND - d_month_seq BETWEEN 1212 AND 1212 + 11) y, - (SELECT - s_store_name s_store_name2, - wss.d_week_seq d_week_seq2, - s_store_id s_store_id2, - sun_sales sun_sales2, - mon_sales mon_sales2, - tue_sales tue_sales2, - wed_sales wed_sales2, - thu_sales thu_sales2, - fri_sales fri_sales2, - sat_sales sat_sales2 - FROM wss, store, date_dim d - WHERE d.d_week_seq = wss.d_week_seq AND - ss_store_sk = s_store_sk AND - d_month_seq BETWEEN 1212 + 12 AND 1212 + 23) x -WHERE s_store_id1 = s_store_id2 - AND d_week_seq1 = d_week_seq2 - 52 -ORDER BY s_store_name1, s_store_id1, d_week_seq1 -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q6.sql b/src/test/resources/tpcds/queries/q6.sql deleted file mode 100644 index f0f5cf05a..000000000 --- a/src/test/resources/tpcds/queries/q6.sql +++ /dev/null @@ -1,21 +0,0 @@ -SELECT - a.ca_state state, - count(*) cnt -FROM - customer_address a, customer c, store_sales s, date_dim d, item i -WHERE a.ca_address_sk = c.c_current_addr_sk - AND c.c_customer_sk = s.ss_customer_sk - AND s.ss_sold_date_sk = d.d_date_sk - AND s.ss_item_sk = i.i_item_sk - AND d.d_month_seq = - (SELECT DISTINCT (d_month_seq) - FROM date_dim - WHERE d_year = 2000 AND d_moy = 1) - AND i.i_current_price > 1.2 * - (SELECT avg(j.i_current_price) - FROM item j - WHERE j.i_category = i.i_category) -GROUP BY a.ca_state -HAVING count(*) >= 10 -ORDER BY cnt -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q60.sql b/src/test/resources/tpcds/queries/q60.sql deleted file mode 100644 index 41b963f44..000000000 --- a/src/test/resources/tpcds/queries/q60.sql +++ /dev/null @@ -1,62 +0,0 @@ -WITH ss AS ( - SELECT - i_item_id, - sum(ss_ext_sales_price) total_sales - FROM store_sales, date_dim, customer_address, item - WHERE - i_item_id IN (SELECT i_item_id - FROM item - WHERE i_category IN ('Music')) - AND ss_item_sk = i_item_sk - AND ss_sold_date_sk = d_date_sk - AND d_year = 1998 - AND d_moy = 9 - AND ss_addr_sk = ca_address_sk - AND ca_gmt_offset = -5 - GROUP BY i_item_id), - cs AS ( - SELECT - i_item_id, - sum(cs_ext_sales_price) total_sales - FROM catalog_sales, date_dim, customer_address, item - WHERE - i_item_id IN (SELECT i_item_id - FROM item - WHERE i_category IN ('Music')) - AND cs_item_sk = i_item_sk - AND cs_sold_date_sk = d_date_sk - AND d_year = 1998 - AND d_moy = 9 - AND cs_bill_addr_sk = ca_address_sk - AND ca_gmt_offset = -5 - GROUP BY i_item_id), - ws AS ( - SELECT - i_item_id, - sum(ws_ext_sales_price) total_sales - FROM web_sales, date_dim, customer_address, item - WHERE - i_item_id IN (SELECT i_item_id - FROM item - WHERE i_category IN ('Music')) - AND ws_item_sk = i_item_sk - AND ws_sold_date_sk = d_date_sk - AND d_year = 1998 - AND d_moy = 9 - AND ws_bill_addr_sk = ca_address_sk - AND ca_gmt_offset = -5 - GROUP BY i_item_id) -SELECT - i_item_id, - sum(total_sales) total_sales -FROM (SELECT * - FROM ss - UNION ALL - SELECT * - FROM cs - UNION ALL - SELECT * - FROM ws) tmp1 -GROUP BY i_item_id -ORDER BY i_item_id, total_sales -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q61.sql b/src/test/resources/tpcds/queries/q61.sql deleted file mode 100644 index b0a872b4b..000000000 --- a/src/test/resources/tpcds/queries/q61.sql +++ /dev/null @@ -1,33 +0,0 @@ -SELECT - promotions, - total, - cast(promotions AS DECIMAL(15, 4)) / cast(total AS DECIMAL(15, 4)) * 100 -FROM - (SELECT sum(ss_ext_sales_price) promotions - FROM store_sales, store, promotion, date_dim, customer, customer_address, item - WHERE ss_sold_date_sk = d_date_sk - AND ss_store_sk = s_store_sk - AND ss_promo_sk = p_promo_sk - AND ss_customer_sk = c_customer_sk - AND ca_address_sk = c_current_addr_sk - AND ss_item_sk = i_item_sk - AND ca_gmt_offset = -5 - AND i_category = 'Jewelry' - AND (p_channel_dmail = 'Y' OR p_channel_email = 'Y' OR p_channel_tv = 'Y') - AND s_gmt_offset = -5 - AND d_year = 1998 - AND d_moy = 11) promotional_sales, - (SELECT sum(ss_ext_sales_price) total - FROM store_sales, store, date_dim, customer, customer_address, item - WHERE ss_sold_date_sk = d_date_sk - AND ss_store_sk = s_store_sk - AND ss_customer_sk = c_customer_sk - AND ca_address_sk = c_current_addr_sk - AND ss_item_sk = i_item_sk - AND ca_gmt_offset = -5 - AND i_category = 'Jewelry' - AND s_gmt_offset = -5 - AND d_year = 1998 - AND d_moy = 11) all_sales -ORDER BY promotions, total -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q62.sql b/src/test/resources/tpcds/queries/q62.sql deleted file mode 100644 index 8a414f154..000000000 --- a/src/test/resources/tpcds/queries/q62.sql +++ /dev/null @@ -1,35 +0,0 @@ -SELECT - substr(w_warehouse_name, 1, 20), - sm_type, - web_name, - sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk <= 30) - THEN 1 - ELSE 0 END) AS `30 days `, - sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 30) AND - (ws_ship_date_sk - ws_sold_date_sk <= 60) - THEN 1 - ELSE 0 END) AS `31 - 60 days `, - sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 60) AND - (ws_ship_date_sk - ws_sold_date_sk <= 90) - THEN 1 - ELSE 0 END) AS `61 - 90 days `, - sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 90) AND - (ws_ship_date_sk - ws_sold_date_sk <= 120) - THEN 1 - ELSE 0 END) AS `91 - 120 days `, - sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 120) - THEN 1 - ELSE 0 END) AS `>120 days ` -FROM - web_sales, warehouse, ship_mode, web_site, date_dim -WHERE - d_month_seq BETWEEN 1200 AND 1200 + 11 - AND ws_ship_date_sk = d_date_sk - AND ws_warehouse_sk = w_warehouse_sk - AND ws_ship_mode_sk = sm_ship_mode_sk - AND ws_web_site_sk = web_site_sk -GROUP BY - substr(w_warehouse_name, 1, 20), sm_type, web_name -ORDER BY - substr(w_warehouse_name, 1, 20), sm_type, web_name -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q63.sql b/src/test/resources/tpcds/queries/q63.sql deleted file mode 100644 index ef6867e0a..000000000 --- a/src/test/resources/tpcds/queries/q63.sql +++ /dev/null @@ -1,31 +0,0 @@ -SELECT * -FROM (SELECT - i_manager_id, - sum(ss_sales_price) sum_sales, - avg(sum(ss_sales_price)) - OVER (PARTITION BY i_manager_id) avg_monthly_sales -FROM item - , store_sales - , date_dim - , store -WHERE ss_item_sk = i_item_sk - AND ss_sold_date_sk = d_date_sk - AND ss_store_sk = s_store_sk - AND d_month_seq IN (1200, 1200 + 1, 1200 + 2, 1200 + 3, 1200 + 4, 1200 + 5, 1200 + 6, 1200 + 7, - 1200 + 8, 1200 + 9, 1200 + 10, 1200 + 11) - AND ((i_category IN ('Books', 'Children', 'Electronics') - AND i_class IN ('personal', 'portable', 'refernece', 'self-help') - AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', - 'exportiunivamalg #9', 'scholaramalgamalg #9')) - OR (i_category IN ('Women', 'Music', 'Men') - AND i_class IN ('accessories', 'classical', 'fragrances', 'pants') - AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', - 'importoamalg #1'))) -GROUP BY i_manager_id, d_moy) tmp1 -WHERE CASE WHEN avg_monthly_sales > 0 - THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales - ELSE NULL END > 0.1 -ORDER BY i_manager_id - , avg_monthly_sales - , sum_sales -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q64.sql b/src/test/resources/tpcds/queries/q64.sql deleted file mode 100644 index 8ec1d31b6..000000000 --- a/src/test/resources/tpcds/queries/q64.sql +++ /dev/null @@ -1,92 +0,0 @@ -WITH cs_ui AS -(SELECT - cs_item_sk, - sum(cs_ext_list_price) AS sale, - sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit) AS refund - FROM catalog_sales - , catalog_returns - WHERE cs_item_sk = cr_item_sk - AND cs_order_number = cr_order_number - GROUP BY cs_item_sk - HAVING sum(cs_ext_list_price) > 2 * sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit)), - cross_sales AS - (SELECT - i_product_name product_name, - i_item_sk item_sk, - s_store_name store_name, - s_zip store_zip, - ad1.ca_street_number b_street_number, - ad1.ca_street_name b_streen_name, - ad1.ca_city b_city, - ad1.ca_zip b_zip, - ad2.ca_street_number c_street_number, - ad2.ca_street_name c_street_name, - ad2.ca_city c_city, - ad2.ca_zip c_zip, - d1.d_year AS syear, - d2.d_year AS fsyear, - d3.d_year s2year, - count(*) cnt, - sum(ss_wholesale_cost) s1, - sum(ss_list_price) s2, - sum(ss_coupon_amt) s3 - FROM store_sales, store_returns, cs_ui, date_dim d1, date_dim d2, date_dim d3, - store, customer, customer_demographics cd1, customer_demographics cd2, - promotion, household_demographics hd1, household_demographics hd2, - customer_address ad1, customer_address ad2, income_band ib1, income_band ib2, item - WHERE ss_store_sk = s_store_sk AND - ss_sold_date_sk = d1.d_date_sk AND - ss_customer_sk = c_customer_sk AND - ss_cdemo_sk = cd1.cd_demo_sk AND - ss_hdemo_sk = hd1.hd_demo_sk AND - ss_addr_sk = ad1.ca_address_sk AND - ss_item_sk = i_item_sk AND - ss_item_sk = sr_item_sk AND - ss_ticket_number = sr_ticket_number AND - ss_item_sk = cs_ui.cs_item_sk AND - c_current_cdemo_sk = cd2.cd_demo_sk AND - c_current_hdemo_sk = hd2.hd_demo_sk AND - c_current_addr_sk = ad2.ca_address_sk AND - c_first_sales_date_sk = d2.d_date_sk AND - c_first_shipto_date_sk = d3.d_date_sk AND - ss_promo_sk = p_promo_sk AND - hd1.hd_income_band_sk = ib1.ib_income_band_sk AND - hd2.hd_income_band_sk = ib2.ib_income_band_sk AND - cd1.cd_marital_status <> cd2.cd_marital_status AND - i_color IN ('purple', 'burlywood', 'indian', 'spring', 'floral', 'medium') AND - i_current_price BETWEEN 64 AND 64 + 10 AND - i_current_price BETWEEN 64 + 1 AND 64 + 15 - GROUP BY i_product_name, i_item_sk, s_store_name, s_zip, ad1.ca_street_number, - ad1.ca_street_name, ad1.ca_city, ad1.ca_zip, ad2.ca_street_number, - ad2.ca_street_name, ad2.ca_city, ad2.ca_zip, d1.d_year, d2.d_year, d3.d_year - ) -SELECT - cs1.product_name, - cs1.store_name, - cs1.store_zip, - cs1.b_street_number, - cs1.b_streen_name, - cs1.b_city, - cs1.b_zip, - cs1.c_street_number, - cs1.c_street_name, - cs1.c_city, - cs1.c_zip, - cs1.syear, - cs1.cnt, - cs1.s1, - cs1.s2, - cs1.s3, - cs2.s1, - cs2.s2, - cs2.s3, - cs2.syear, - cs2.cnt -FROM cross_sales cs1, cross_sales cs2 -WHERE cs1.item_sk = cs2.item_sk AND - cs1.syear = 1999 AND - cs2.syear = 1999 + 1 AND - cs2.cnt <= cs1.cnt AND - cs1.store_name = cs2.store_name AND - cs1.store_zip = cs2.store_zip -ORDER BY cs1.product_name, cs1.store_name, cs2.cnt diff --git a/src/test/resources/tpcds/queries/q65.sql b/src/test/resources/tpcds/queries/q65.sql deleted file mode 100644 index aad04be1b..000000000 --- a/src/test/resources/tpcds/queries/q65.sql +++ /dev/null @@ -1,33 +0,0 @@ -SELECT - s_store_name, - i_item_desc, - sc.revenue, - i_current_price, - i_wholesale_cost, - i_brand -FROM store, item, - (SELECT - ss_store_sk, - avg(revenue) AS ave - FROM - (SELECT - ss_store_sk, - ss_item_sk, - sum(ss_sales_price) AS revenue - FROM store_sales, date_dim - WHERE ss_sold_date_sk = d_date_sk AND d_month_seq BETWEEN 1176 AND 1176 + 11 - GROUP BY ss_store_sk, ss_item_sk) sa - GROUP BY ss_store_sk) sb, - (SELECT - ss_store_sk, - ss_item_sk, - sum(ss_sales_price) AS revenue - FROM store_sales, date_dim - WHERE ss_sold_date_sk = d_date_sk AND d_month_seq BETWEEN 1176 AND 1176 + 11 - GROUP BY ss_store_sk, ss_item_sk) sc -WHERE sb.ss_store_sk = sc.ss_store_sk AND - sc.revenue <= 0.1 * sb.ave AND - s_store_sk = sc.ss_store_sk AND - i_item_sk = sc.ss_item_sk -ORDER BY s_store_name, i_item_desc -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q66.sql b/src/test/resources/tpcds/queries/q66.sql deleted file mode 100644 index f826b4164..000000000 --- a/src/test/resources/tpcds/queries/q66.sql +++ /dev/null @@ -1,240 +0,0 @@ -SELECT - w_warehouse_name, - w_warehouse_sq_ft, - w_city, - w_county, - w_state, - w_country, - ship_carriers, - year, - sum(jan_sales) AS jan_sales, - sum(feb_sales) AS feb_sales, - sum(mar_sales) AS mar_sales, - sum(apr_sales) AS apr_sales, - sum(may_sales) AS may_sales, - sum(jun_sales) AS jun_sales, - sum(jul_sales) AS jul_sales, - sum(aug_sales) AS aug_sales, - sum(sep_sales) AS sep_sales, - sum(oct_sales) AS oct_sales, - sum(nov_sales) AS nov_sales, - sum(dec_sales) AS dec_sales, - sum(jan_sales / w_warehouse_sq_ft) AS jan_sales_per_sq_foot, - sum(feb_sales / w_warehouse_sq_ft) AS feb_sales_per_sq_foot, - sum(mar_sales / w_warehouse_sq_ft) AS mar_sales_per_sq_foot, - sum(apr_sales / w_warehouse_sq_ft) AS apr_sales_per_sq_foot, - sum(may_sales / w_warehouse_sq_ft) AS may_sales_per_sq_foot, - sum(jun_sales / w_warehouse_sq_ft) AS jun_sales_per_sq_foot, - sum(jul_sales / w_warehouse_sq_ft) AS jul_sales_per_sq_foot, - sum(aug_sales / w_warehouse_sq_ft) AS aug_sales_per_sq_foot, - sum(sep_sales / w_warehouse_sq_ft) AS sep_sales_per_sq_foot, - sum(oct_sales / w_warehouse_sq_ft) AS oct_sales_per_sq_foot, - sum(nov_sales / w_warehouse_sq_ft) AS nov_sales_per_sq_foot, - sum(dec_sales / w_warehouse_sq_ft) AS dec_sales_per_sq_foot, - sum(jan_net) AS jan_net, - sum(feb_net) AS feb_net, - sum(mar_net) AS mar_net, - sum(apr_net) AS apr_net, - sum(may_net) AS may_net, - sum(jun_net) AS jun_net, - sum(jul_net) AS jul_net, - sum(aug_net) AS aug_net, - sum(sep_net) AS sep_net, - sum(oct_net) AS oct_net, - sum(nov_net) AS nov_net, - sum(dec_net) AS dec_net -FROM ( - (SELECT - w_warehouse_name, - w_warehouse_sq_ft, - w_city, - w_county, - w_state, - w_country, - concat('DHL', ',', 'BARIAN') AS ship_carriers, - d_year AS year, - sum(CASE WHEN d_moy = 1 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS jan_sales, - sum(CASE WHEN d_moy = 2 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS feb_sales, - sum(CASE WHEN d_moy = 3 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS mar_sales, - sum(CASE WHEN d_moy = 4 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS apr_sales, - sum(CASE WHEN d_moy = 5 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS may_sales, - sum(CASE WHEN d_moy = 6 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS jun_sales, - sum(CASE WHEN d_moy = 7 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS jul_sales, - sum(CASE WHEN d_moy = 8 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS aug_sales, - sum(CASE WHEN d_moy = 9 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS sep_sales, - sum(CASE WHEN d_moy = 10 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS oct_sales, - sum(CASE WHEN d_moy = 11 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS nov_sales, - sum(CASE WHEN d_moy = 12 - THEN ws_ext_sales_price * ws_quantity - ELSE 0 END) AS dec_sales, - sum(CASE WHEN d_moy = 1 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS jan_net, - sum(CASE WHEN d_moy = 2 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS feb_net, - sum(CASE WHEN d_moy = 3 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS mar_net, - sum(CASE WHEN d_moy = 4 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS apr_net, - sum(CASE WHEN d_moy = 5 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS may_net, - sum(CASE WHEN d_moy = 6 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS jun_net, - sum(CASE WHEN d_moy = 7 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS jul_net, - sum(CASE WHEN d_moy = 8 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS aug_net, - sum(CASE WHEN d_moy = 9 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS sep_net, - sum(CASE WHEN d_moy = 10 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS oct_net, - sum(CASE WHEN d_moy = 11 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS nov_net, - sum(CASE WHEN d_moy = 12 - THEN ws_net_paid * ws_quantity - ELSE 0 END) AS dec_net - FROM - web_sales, warehouse, date_dim, time_dim, ship_mode - WHERE - ws_warehouse_sk = w_warehouse_sk - AND ws_sold_date_sk = d_date_sk - AND ws_sold_time_sk = t_time_sk - AND ws_ship_mode_sk = sm_ship_mode_sk - AND d_year = 2001 - AND t_time BETWEEN 30838 AND 30838 + 28800 - AND sm_carrier IN ('DHL', 'BARIAN') - GROUP BY - w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, d_year) - UNION ALL - (SELECT - w_warehouse_name, - w_warehouse_sq_ft, - w_city, - w_county, - w_state, - w_country, - concat('DHL', ',', 'BARIAN') AS ship_carriers, - d_year AS year, - sum(CASE WHEN d_moy = 1 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS jan_sales, - sum(CASE WHEN d_moy = 2 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS feb_sales, - sum(CASE WHEN d_moy = 3 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS mar_sales, - sum(CASE WHEN d_moy = 4 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS apr_sales, - sum(CASE WHEN d_moy = 5 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS may_sales, - sum(CASE WHEN d_moy = 6 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS jun_sales, - sum(CASE WHEN d_moy = 7 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS jul_sales, - sum(CASE WHEN d_moy = 8 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS aug_sales, - sum(CASE WHEN d_moy = 9 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS sep_sales, - sum(CASE WHEN d_moy = 10 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS oct_sales, - sum(CASE WHEN d_moy = 11 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS nov_sales, - sum(CASE WHEN d_moy = 12 - THEN cs_sales_price * cs_quantity - ELSE 0 END) AS dec_sales, - sum(CASE WHEN d_moy = 1 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS jan_net, - sum(CASE WHEN d_moy = 2 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS feb_net, - sum(CASE WHEN d_moy = 3 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS mar_net, - sum(CASE WHEN d_moy = 4 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS apr_net, - sum(CASE WHEN d_moy = 5 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS may_net, - sum(CASE WHEN d_moy = 6 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS jun_net, - sum(CASE WHEN d_moy = 7 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS jul_net, - sum(CASE WHEN d_moy = 8 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS aug_net, - sum(CASE WHEN d_moy = 9 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS sep_net, - sum(CASE WHEN d_moy = 10 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS oct_net, - sum(CASE WHEN d_moy = 11 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS nov_net, - sum(CASE WHEN d_moy = 12 - THEN cs_net_paid_inc_tax * cs_quantity - ELSE 0 END) AS dec_net - FROM - catalog_sales, warehouse, date_dim, time_dim, ship_mode - WHERE - cs_warehouse_sk = w_warehouse_sk - AND cs_sold_date_sk = d_date_sk - AND cs_sold_time_sk = t_time_sk - AND cs_ship_mode_sk = sm_ship_mode_sk - AND d_year = 2001 - AND t_time BETWEEN 30838 AND 30838 + 28800 - AND sm_carrier IN ('DHL', 'BARIAN') - GROUP BY - w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, d_year - ) - ) x -GROUP BY - w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, - ship_carriers, year -ORDER BY w_warehouse_name -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q67.sql b/src/test/resources/tpcds/queries/q67.sql deleted file mode 100644 index f66e2252b..000000000 --- a/src/test/resources/tpcds/queries/q67.sql +++ /dev/null @@ -1,38 +0,0 @@ -SELECT * -FROM - (SELECT - i_category, - i_class, - i_brand, - i_product_name, - d_year, - d_qoy, - d_moy, - s_store_id, - sumsales, - rank() - OVER (PARTITION BY i_category - ORDER BY sumsales DESC) rk - FROM - (SELECT - i_category, - i_class, - i_brand, - i_product_name, - d_year, - d_qoy, - d_moy, - s_store_id, - sum(coalesce(ss_sales_price * ss_quantity, 0)) sumsales - FROM store_sales, date_dim, store, item - WHERE ss_sold_date_sk = d_date_sk - AND ss_item_sk = i_item_sk - AND ss_store_sk = s_store_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11 - GROUP BY ROLLUP (i_category, i_class, i_brand, i_product_name, d_year, d_qoy, - d_moy, s_store_id)) dw1) dw2 -WHERE rk <= 100 -ORDER BY - i_category, i_class, i_brand, i_product_name, d_year, - d_qoy, d_moy, s_store_id, sumsales, rk -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q68.sql b/src/test/resources/tpcds/queries/q68.sql deleted file mode 100644 index adb8a7189..000000000 --- a/src/test/resources/tpcds/queries/q68.sql +++ /dev/null @@ -1,34 +0,0 @@ -SELECT - c_last_name, - c_first_name, - ca_city, - bought_city, - ss_ticket_number, - extended_price, - extended_tax, - list_price -FROM (SELECT - ss_ticket_number, - ss_customer_sk, - ca_city bought_city, - sum(ss_ext_sales_price) extended_price, - sum(ss_ext_list_price) list_price, - sum(ss_ext_tax) extended_tax -FROM store_sales, date_dim, store, household_demographics, customer_address -WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk - AND store_sales.ss_store_sk = store.s_store_sk - AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk - AND store_sales.ss_addr_sk = customer_address.ca_address_sk - AND date_dim.d_dom BETWEEN 1 AND 2 - AND (household_demographics.hd_dep_count = 4 OR - household_demographics.hd_vehicle_count = 3) - AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) - AND store.s_city IN ('Midway', 'Fairview') -GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city) dn, - customer, - customer_address current_addr -WHERE ss_customer_sk = c_customer_sk - AND customer.c_current_addr_sk = current_addr.ca_address_sk - AND current_addr.ca_city <> bought_city -ORDER BY c_last_name, ss_ticket_number -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q69.sql b/src/test/resources/tpcds/queries/q69.sql deleted file mode 100644 index 1f0ee64f5..000000000 --- a/src/test/resources/tpcds/queries/q69.sql +++ /dev/null @@ -1,38 +0,0 @@ -SELECT - cd_gender, - cd_marital_status, - cd_education_status, - count(*) cnt1, - cd_purchase_estimate, - count(*) cnt2, - cd_credit_rating, - count(*) cnt3 -FROM - customer c, customer_address ca, customer_demographics -WHERE - c.c_current_addr_sk = ca.ca_address_sk AND - ca_state IN ('KY', 'GA', 'NM') AND - cd_demo_sk = c.c_current_cdemo_sk AND - exists(SELECT * - FROM store_sales, date_dim - WHERE c.c_customer_sk = ss_customer_sk AND - ss_sold_date_sk = d_date_sk AND - d_year = 2001 AND - d_moy BETWEEN 4 AND 4 + 2) AND - (NOT exists(SELECT * - FROM web_sales, date_dim - WHERE c.c_customer_sk = ws_bill_customer_sk AND - ws_sold_date_sk = d_date_sk AND - d_year = 2001 AND - d_moy BETWEEN 4 AND 4 + 2) AND - NOT exists(SELECT * - FROM catalog_sales, date_dim - WHERE c.c_customer_sk = cs_ship_customer_sk AND - cs_sold_date_sk = d_date_sk AND - d_year = 2001 AND - d_moy BETWEEN 4 AND 4 + 2)) -GROUP BY cd_gender, cd_marital_status, cd_education_status, - cd_purchase_estimate, cd_credit_rating -ORDER BY cd_gender, cd_marital_status, cd_education_status, - cd_purchase_estimate, cd_credit_rating -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q7.sql b/src/test/resources/tpcds/queries/q7.sql deleted file mode 100644 index 6630a0054..000000000 --- a/src/test/resources/tpcds/queries/q7.sql +++ /dev/null @@ -1,19 +0,0 @@ -SELECT - i_item_id, - avg(ss_quantity) agg1, - avg(ss_list_price) agg2, - avg(ss_coupon_amt) agg3, - avg(ss_sales_price) agg4 -FROM store_sales, customer_demographics, date_dim, item, promotion -WHERE ss_sold_date_sk = d_date_sk AND - ss_item_sk = i_item_sk AND - ss_cdemo_sk = cd_demo_sk AND - ss_promo_sk = p_promo_sk AND - cd_gender = 'M' AND - cd_marital_status = 'S' AND - cd_education_status = 'College' AND - (p_channel_email = 'N' OR p_channel_event = 'N') AND - d_year = 2000 -GROUP BY i_item_id -ORDER BY i_item_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q70.sql b/src/test/resources/tpcds/queries/q70.sql deleted file mode 100644 index 625011b21..000000000 --- a/src/test/resources/tpcds/queries/q70.sql +++ /dev/null @@ -1,38 +0,0 @@ -SELECT - sum(ss_net_profit) AS total_sum, - s_state, - s_county, - grouping(s_state) + grouping(s_county) AS lochierarchy, - rank() - OVER ( - PARTITION BY grouping(s_state) + grouping(s_county), - CASE WHEN grouping(s_county) = 0 - THEN s_state END - ORDER BY sum(ss_net_profit) DESC) AS rank_within_parent -FROM - store_sales, date_dim d1, store -WHERE - d1.d_month_seq BETWEEN 1200 AND 1200 + 11 - AND d1.d_date_sk = ss_sold_date_sk - AND s_store_sk = ss_store_sk - AND s_state IN - (SELECT s_state - FROM - (SELECT - s_state AS s_state, - rank() - OVER (PARTITION BY s_state - ORDER BY sum(ss_net_profit) DESC) AS ranking - FROM store_sales, store, date_dim - WHERE d_month_seq BETWEEN 1200 AND 1200 + 11 - AND d_date_sk = ss_sold_date_sk - AND s_store_sk = ss_store_sk - GROUP BY s_state) tmp1 - WHERE ranking <= 5) -GROUP BY ROLLUP (s_state, s_county) -ORDER BY - lochierarchy DESC - , CASE WHEN lochierarchy = 0 - THEN s_state END - , rank_within_parent -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q71.sql b/src/test/resources/tpcds/queries/q71.sql deleted file mode 100644 index 8d724b924..000000000 --- a/src/test/resources/tpcds/queries/q71.sql +++ /dev/null @@ -1,44 +0,0 @@ -SELECT - i_brand_id brand_id, - i_brand brand, - t_hour, - t_minute, - sum(ext_price) ext_price -FROM item, - (SELECT - ws_ext_sales_price AS ext_price, - ws_sold_date_sk AS sold_date_sk, - ws_item_sk AS sold_item_sk, - ws_sold_time_sk AS time_sk - FROM web_sales, date_dim - WHERE d_date_sk = ws_sold_date_sk - AND d_moy = 11 - AND d_year = 1999 - UNION ALL - SELECT - cs_ext_sales_price AS ext_price, - cs_sold_date_sk AS sold_date_sk, - cs_item_sk AS sold_item_sk, - cs_sold_time_sk AS time_sk - FROM catalog_sales, date_dim - WHERE d_date_sk = cs_sold_date_sk - AND d_moy = 11 - AND d_year = 1999 - UNION ALL - SELECT - ss_ext_sales_price AS ext_price, - ss_sold_date_sk AS sold_date_sk, - ss_item_sk AS sold_item_sk, - ss_sold_time_sk AS time_sk - FROM store_sales, date_dim - WHERE d_date_sk = ss_sold_date_sk - AND d_moy = 11 - AND d_year = 1999 - ) AS tmp, time_dim -WHERE - sold_item_sk = i_item_sk - AND i_manager_id = 1 - AND time_sk = t_time_sk - AND (t_meal_time = 'breakfast' OR t_meal_time = 'dinner') -GROUP BY i_brand, i_brand_id, t_hour, t_minute -ORDER BY ext_price DESC, brand_id diff --git a/src/test/resources/tpcds/queries/q72.sql b/src/test/resources/tpcds/queries/q72.sql deleted file mode 100644 index 99b3eee54..000000000 --- a/src/test/resources/tpcds/queries/q72.sql +++ /dev/null @@ -1,33 +0,0 @@ -SELECT - i_item_desc, - w_warehouse_name, - d1.d_week_seq, - count(CASE WHEN p_promo_sk IS NULL - THEN 1 - ELSE 0 END) no_promo, - count(CASE WHEN p_promo_sk IS NOT NULL - THEN 1 - ELSE 0 END) promo, - count(*) total_cnt -FROM catalog_sales - JOIN inventory ON (cs_item_sk = inv_item_sk) - JOIN warehouse ON (w_warehouse_sk = inv_warehouse_sk) - JOIN item ON (i_item_sk = cs_item_sk) - JOIN customer_demographics ON (cs_bill_cdemo_sk = cd_demo_sk) - JOIN household_demographics ON (cs_bill_hdemo_sk = hd_demo_sk) - JOIN date_dim d1 ON (cs_sold_date_sk = d1.d_date_sk) - JOIN date_dim d2 ON (inv_date_sk = d2.d_date_sk) - JOIN date_dim d3 ON (cs_ship_date_sk = d3.d_date_sk) - LEFT OUTER JOIN promotion ON (cs_promo_sk = p_promo_sk) - LEFT OUTER JOIN catalog_returns ON (cr_item_sk = cs_item_sk AND cr_order_number = cs_order_number) -WHERE d1.d_week_seq = d2.d_week_seq - AND inv_quantity_on_hand < cs_quantity - AND d3.d_date > (cast(d1.d_date AS DATE) + interval 5 days) - AND hd_buy_potential = '>10000' - AND d1.d_year = 1999 - AND hd_buy_potential = '>10000' - AND cd_marital_status = 'D' - AND d1.d_year = 1999 -GROUP BY i_item_desc, w_warehouse_name, d1.d_week_seq -ORDER BY total_cnt DESC, i_item_desc, w_warehouse_name, d_week_seq -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q73.sql b/src/test/resources/tpcds/queries/q73.sql deleted file mode 100644 index 881be2e90..000000000 --- a/src/test/resources/tpcds/queries/q73.sql +++ /dev/null @@ -1,30 +0,0 @@ -SELECT - c_last_name, - c_first_name, - c_salutation, - c_preferred_cust_flag, - ss_ticket_number, - cnt -FROM - (SELECT - ss_ticket_number, - ss_customer_sk, - count(*) cnt - FROM store_sales, date_dim, store, household_demographics - WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk - AND store_sales.ss_store_sk = store.s_store_sk - AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk - AND date_dim.d_dom BETWEEN 1 AND 2 - AND (household_demographics.hd_buy_potential = '>10000' OR - household_demographics.hd_buy_potential = 'unknown') - AND household_demographics.hd_vehicle_count > 0 - AND CASE WHEN household_demographics.hd_vehicle_count > 0 - THEN - household_demographics.hd_dep_count / household_demographics.hd_vehicle_count - ELSE NULL END > 1 - AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) - AND store.s_county IN ('Williamson County', 'Franklin Parish', 'Bronx County', 'Orange County') - GROUP BY ss_ticket_number, ss_customer_sk) dj, customer -WHERE ss_customer_sk = c_customer_sk - AND cnt BETWEEN 1 AND 5 -ORDER BY cnt DESC diff --git a/src/test/resources/tpcds/queries/q74.sql b/src/test/resources/tpcds/queries/q74.sql deleted file mode 100644 index 154b26d68..000000000 --- a/src/test/resources/tpcds/queries/q74.sql +++ /dev/null @@ -1,58 +0,0 @@ -WITH year_total AS ( - SELECT - c_customer_id customer_id, - c_first_name customer_first_name, - c_last_name customer_last_name, - d_year AS year, - sum(ss_net_paid) year_total, - 's' sale_type - FROM - customer, store_sales, date_dim - WHERE c_customer_sk = ss_customer_sk - AND ss_sold_date_sk = d_date_sk - AND d_year IN (2001, 2001 + 1) - GROUP BY - c_customer_id, c_first_name, c_last_name, d_year - UNION ALL - SELECT - c_customer_id customer_id, - c_first_name customer_first_name, - c_last_name customer_last_name, - d_year AS year, - sum(ws_net_paid) year_total, - 'w' sale_type - FROM - customer, web_sales, date_dim - WHERE c_customer_sk = ws_bill_customer_sk - AND ws_sold_date_sk = d_date_sk - AND d_year IN (2001, 2001 + 1) - GROUP BY - c_customer_id, c_first_name, c_last_name, d_year) -SELECT - t_s_secyear.customer_id, - t_s_secyear.customer_first_name, - t_s_secyear.customer_last_name -FROM - year_total t_s_firstyear, year_total t_s_secyear, - year_total t_w_firstyear, year_total t_w_secyear -WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id - AND t_s_firstyear.customer_id = t_w_secyear.customer_id - AND t_s_firstyear.customer_id = t_w_firstyear.customer_id - AND t_s_firstyear.sale_type = 's' - AND t_w_firstyear.sale_type = 'w' - AND t_s_secyear.sale_type = 's' - AND t_w_secyear.sale_type = 'w' - AND t_s_firstyear.year = 2001 - AND t_s_secyear.year = 2001 + 1 - AND t_w_firstyear.year = 2001 - AND t_w_secyear.year = 2001 + 1 - AND t_s_firstyear.year_total > 0 - AND t_w_firstyear.year_total > 0 - AND CASE WHEN t_w_firstyear.year_total > 0 - THEN t_w_secyear.year_total / t_w_firstyear.year_total - ELSE NULL END - > CASE WHEN t_s_firstyear.year_total > 0 - THEN t_s_secyear.year_total / t_s_firstyear.year_total - ELSE NULL END -ORDER BY 1, 1, 1 -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q75.sql b/src/test/resources/tpcds/queries/q75.sql deleted file mode 100644 index 2a143232b..000000000 --- a/src/test/resources/tpcds/queries/q75.sql +++ /dev/null @@ -1,76 +0,0 @@ -WITH all_sales AS ( - SELECT - d_year, - i_brand_id, - i_class_id, - i_category_id, - i_manufact_id, - SUM(sales_cnt) AS sales_cnt, - SUM(sales_amt) AS sales_amt - FROM ( - SELECT - d_year, - i_brand_id, - i_class_id, - i_category_id, - i_manufact_id, - cs_quantity - COALESCE(cr_return_quantity, 0) AS sales_cnt, - cs_ext_sales_price - COALESCE(cr_return_amount, 0.0) AS sales_amt - FROM catalog_sales - JOIN item ON i_item_sk = cs_item_sk - JOIN date_dim ON d_date_sk = cs_sold_date_sk - LEFT JOIN catalog_returns ON (cs_order_number = cr_order_number - AND cs_item_sk = cr_item_sk) - WHERE i_category = 'Books' - UNION - SELECT - d_year, - i_brand_id, - i_class_id, - i_category_id, - i_manufact_id, - ss_quantity - COALESCE(sr_return_quantity, 0) AS sales_cnt, - ss_ext_sales_price - COALESCE(sr_return_amt, 0.0) AS sales_amt - FROM store_sales - JOIN item ON i_item_sk = ss_item_sk - JOIN date_dim ON d_date_sk = ss_sold_date_sk - LEFT JOIN store_returns ON (ss_ticket_number = sr_ticket_number - AND ss_item_sk = sr_item_sk) - WHERE i_category = 'Books' - UNION - SELECT - d_year, - i_brand_id, - i_class_id, - i_category_id, - i_manufact_id, - ws_quantity - COALESCE(wr_return_quantity, 0) AS sales_cnt, - ws_ext_sales_price - COALESCE(wr_return_amt, 0.0) AS sales_amt - FROM web_sales - JOIN item ON i_item_sk = ws_item_sk - JOIN date_dim ON d_date_sk = ws_sold_date_sk - LEFT JOIN web_returns ON (ws_order_number = wr_order_number - AND ws_item_sk = wr_item_sk) - WHERE i_category = 'Books') sales_detail - GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) -SELECT - prev_yr.d_year AS prev_year, - curr_yr.d_year AS year, - curr_yr.i_brand_id, - curr_yr.i_class_id, - curr_yr.i_category_id, - curr_yr.i_manufact_id, - prev_yr.sales_cnt AS prev_yr_cnt, - curr_yr.sales_cnt AS curr_yr_cnt, - curr_yr.sales_cnt - prev_yr.sales_cnt AS sales_cnt_diff, - curr_yr.sales_amt - prev_yr.sales_amt AS sales_amt_diff -FROM all_sales curr_yr, all_sales prev_yr -WHERE curr_yr.i_brand_id = prev_yr.i_brand_id - AND curr_yr.i_class_id = prev_yr.i_class_id - AND curr_yr.i_category_id = prev_yr.i_category_id - AND curr_yr.i_manufact_id = prev_yr.i_manufact_id - AND curr_yr.d_year = 2002 - AND prev_yr.d_year = 2002 - 1 - AND CAST(curr_yr.sales_cnt AS DECIMAL(17, 2)) / CAST(prev_yr.sales_cnt AS DECIMAL(17, 2)) < 0.9 -ORDER BY sales_cnt_diff -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q76.sql b/src/test/resources/tpcds/queries/q76.sql deleted file mode 100644 index 815fa922b..000000000 --- a/src/test/resources/tpcds/queries/q76.sql +++ /dev/null @@ -1,47 +0,0 @@ -SELECT - channel, - col_name, - d_year, - d_qoy, - i_category, - COUNT(*) sales_cnt, - SUM(ext_sales_price) sales_amt -FROM ( - SELECT - 'store' AS channel, - ss_store_sk col_name, - d_year, - d_qoy, - i_category, - ss_ext_sales_price ext_sales_price - FROM store_sales, item, date_dim - WHERE ss_store_sk IS NULL - AND ss_sold_date_sk = d_date_sk - AND ss_item_sk = i_item_sk - UNION ALL - SELECT - 'web' AS channel, - ws_ship_customer_sk col_name, - d_year, - d_qoy, - i_category, - ws_ext_sales_price ext_sales_price - FROM web_sales, item, date_dim - WHERE ws_ship_customer_sk IS NULL - AND ws_sold_date_sk = d_date_sk - AND ws_item_sk = i_item_sk - UNION ALL - SELECT - 'catalog' AS channel, - cs_ship_addr_sk col_name, - d_year, - d_qoy, - i_category, - cs_ext_sales_price ext_sales_price - FROM catalog_sales, item, date_dim - WHERE cs_ship_addr_sk IS NULL - AND cs_sold_date_sk = d_date_sk - AND cs_item_sk = i_item_sk) foo -GROUP BY channel, col_name, d_year, d_qoy, i_category -ORDER BY channel, col_name, d_year, d_qoy, i_category -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q77.sql b/src/test/resources/tpcds/queries/q77.sql deleted file mode 100644 index a69df9fbc..000000000 --- a/src/test/resources/tpcds/queries/q77.sql +++ /dev/null @@ -1,100 +0,0 @@ -WITH ss AS -(SELECT - s_store_sk, - sum(ss_ext_sales_price) AS sales, - sum(ss_net_profit) AS profit - FROM store_sales, date_dim, store - WHERE ss_sold_date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-03' AS DATE) AND - (cast('2000-08-03' AS DATE) + INTERVAL 30 days) - AND ss_store_sk = s_store_sk - GROUP BY s_store_sk), - sr AS - (SELECT - s_store_sk, - sum(sr_return_amt) AS returns, - sum(sr_net_loss) AS profit_loss - FROM store_returns, date_dim, store - WHERE sr_returned_date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-03' AS DATE) AND - (cast('2000-08-03' AS DATE) + INTERVAL 30 days) - AND sr_store_sk = s_store_sk - GROUP BY s_store_sk), - cs AS - (SELECT - cs_call_center_sk, - sum(cs_ext_sales_price) AS sales, - sum(cs_net_profit) AS profit - FROM catalog_sales, date_dim - WHERE cs_sold_date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-03' AS DATE) AND - (cast('2000-08-03' AS DATE) + INTERVAL 30 days) - GROUP BY cs_call_center_sk), - cr AS - (SELECT - sum(cr_return_amount) AS returns, - sum(cr_net_loss) AS profit_loss - FROM catalog_returns, date_dim - WHERE cr_returned_date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-03' AS DATE) AND - (cast('2000-08-03' AS DATE) + INTERVAL 30 days)), - ws AS - (SELECT - wp_web_page_sk, - sum(ws_ext_sales_price) AS sales, - sum(ws_net_profit) AS profit - FROM web_sales, date_dim, web_page - WHERE ws_sold_date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-03' AS DATE) AND - (cast('2000-08-03' AS DATE) + INTERVAL 30 days) - AND ws_web_page_sk = wp_web_page_sk - GROUP BY wp_web_page_sk), - wr AS - (SELECT - wp_web_page_sk, - sum(wr_return_amt) AS returns, - sum(wr_net_loss) AS profit_loss - FROM web_returns, date_dim, web_page - WHERE wr_returned_date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-03' AS DATE) AND - (cast('2000-08-03' AS DATE) + INTERVAL 30 days) - AND wr_web_page_sk = wp_web_page_sk - GROUP BY wp_web_page_sk) -SELECT - channel, - id, - sum(sales) AS sales, - sum(returns) AS returns, - sum(profit) AS profit -FROM - (SELECT - 'store channel' AS channel, - ss.s_store_sk AS id, - sales, - coalesce(returns, 0) AS returns, - (profit - coalesce(profit_loss, 0)) AS profit - FROM ss - LEFT JOIN sr - ON ss.s_store_sk = sr.s_store_sk - UNION ALL - SELECT - 'catalog channel' AS channel, - cs_call_center_sk AS id, - sales, - returns, - (profit - profit_loss) AS profit - FROM cs, cr - UNION ALL - SELECT - 'web channel' AS channel, - ws.wp_web_page_sk AS id, - sales, - coalesce(returns, 0) returns, - (profit - coalesce(profit_loss, 0)) AS profit - FROM ws - LEFT JOIN wr - ON ws.wp_web_page_sk = wr.wp_web_page_sk - ) x -GROUP BY ROLLUP (channel, id) -ORDER BY channel, id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q78.sql b/src/test/resources/tpcds/queries/q78.sql deleted file mode 100644 index 07b0940e2..000000000 --- a/src/test/resources/tpcds/queries/q78.sql +++ /dev/null @@ -1,64 +0,0 @@ -WITH ws AS -(SELECT - d_year AS ws_sold_year, - ws_item_sk, - ws_bill_customer_sk ws_customer_sk, - sum(ws_quantity) ws_qty, - sum(ws_wholesale_cost) ws_wc, - sum(ws_sales_price) ws_sp - FROM web_sales - LEFT JOIN web_returns ON wr_order_number = ws_order_number AND ws_item_sk = wr_item_sk - JOIN date_dim ON ws_sold_date_sk = d_date_sk - WHERE wr_order_number IS NULL - GROUP BY d_year, ws_item_sk, ws_bill_customer_sk -), - cs AS - (SELECT - d_year AS cs_sold_year, - cs_item_sk, - cs_bill_customer_sk cs_customer_sk, - sum(cs_quantity) cs_qty, - sum(cs_wholesale_cost) cs_wc, - sum(cs_sales_price) cs_sp - FROM catalog_sales - LEFT JOIN catalog_returns ON cr_order_number = cs_order_number AND cs_item_sk = cr_item_sk - JOIN date_dim ON cs_sold_date_sk = d_date_sk - WHERE cr_order_number IS NULL - GROUP BY d_year, cs_item_sk, cs_bill_customer_sk - ), - ss AS - (SELECT - d_year AS ss_sold_year, - ss_item_sk, - ss_customer_sk, - sum(ss_quantity) ss_qty, - sum(ss_wholesale_cost) ss_wc, - sum(ss_sales_price) ss_sp - FROM store_sales - LEFT JOIN store_returns ON sr_ticket_number = ss_ticket_number AND ss_item_sk = sr_item_sk - JOIN date_dim ON ss_sold_date_sk = d_date_sk - WHERE sr_ticket_number IS NULL - GROUP BY d_year, ss_item_sk, ss_customer_sk - ) -SELECT - round(ss_qty / (coalesce(ws_qty + cs_qty, 1)), 2) ratio, - ss_qty store_qty, - ss_wc store_wholesale_cost, - ss_sp store_sales_price, - coalesce(ws_qty, 0) + coalesce(cs_qty, 0) other_chan_qty, - coalesce(ws_wc, 0) + coalesce(cs_wc, 0) other_chan_wholesale_cost, - coalesce(ws_sp, 0) + coalesce(cs_sp, 0) other_chan_sales_price -FROM ss - LEFT JOIN ws - ON (ws_sold_year = ss_sold_year AND ws_item_sk = ss_item_sk AND ws_customer_sk = ss_customer_sk) - LEFT JOIN cs - ON (cs_sold_year = ss_sold_year AND cs_item_sk = ss_item_sk AND cs_customer_sk = ss_customer_sk) -WHERE coalesce(ws_qty, 0) > 0 AND coalesce(cs_qty, 0) > 0 AND ss_sold_year = 2000 -ORDER BY - ratio, - ss_qty DESC, ss_wc DESC, ss_sp DESC, - other_chan_qty, - other_chan_wholesale_cost, - other_chan_sales_price, - round(ss_qty / (coalesce(ws_qty + cs_qty, 1)), 2) -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q79.sql b/src/test/resources/tpcds/queries/q79.sql deleted file mode 100644 index 08f86dc20..000000000 --- a/src/test/resources/tpcds/queries/q79.sql +++ /dev/null @@ -1,27 +0,0 @@ -SELECT - c_last_name, - c_first_name, - substr(s_city, 1, 30), - ss_ticket_number, - amt, - profit -FROM - (SELECT - ss_ticket_number, - ss_customer_sk, - store.s_city, - sum(ss_coupon_amt) amt, - sum(ss_net_profit) profit - FROM store_sales, date_dim, store, household_demographics - WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk - AND store_sales.ss_store_sk = store.s_store_sk - AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk - AND (household_demographics.hd_dep_count = 6 OR - household_demographics.hd_vehicle_count > 2) - AND date_dim.d_dow = 1 - AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) - AND store.s_number_employees BETWEEN 200 AND 295 - GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, store.s_city) ms, customer -WHERE ss_customer_sk = c_customer_sk -ORDER BY c_last_name, c_first_name, substr(s_city, 1, 30), profit -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q8.sql b/src/test/resources/tpcds/queries/q8.sql deleted file mode 100644 index 497725111..000000000 --- a/src/test/resources/tpcds/queries/q8.sql +++ /dev/null @@ -1,87 +0,0 @@ -SELECT - s_store_name, - sum(ss_net_profit) -FROM store_sales, date_dim, store, - (SELECT ca_zip - FROM ( - (SELECT substr(ca_zip, 1, 5) ca_zip - FROM customer_address - WHERE substr(ca_zip, 1, 5) IN ( - '24128','76232','65084','87816','83926','77556','20548', - '26231','43848','15126','91137','61265','98294','25782', - '17920','18426','98235','40081','84093','28577','55565', - '17183','54601','67897','22752','86284','18376','38607', - '45200','21756','29741','96765','23932','89360','29839', - '25989','28898','91068','72550','10390','18845','47770', - '82636','41367','76638','86198','81312','37126','39192', - '88424','72175','81426','53672','10445','42666','66864', - '66708','41248','48583','82276','18842','78890','49448', - '14089','38122','34425','79077','19849','43285','39861', - '66162','77610','13695','99543','83444','83041','12305', - '57665','68341','25003','57834','62878','49130','81096', - '18840','27700','23470','50412','21195','16021','76107', - '71954','68309','18119','98359','64544','10336','86379', - '27068','39736','98569','28915','24206','56529','57647', - '54917','42961','91110','63981','14922','36420','23006', - '67467','32754','30903','20260','31671','51798','72325', - '85816','68621','13955','36446','41766','68806','16725', - '15146','22744','35850','88086','51649','18270','52867', - '39972','96976','63792','11376','94898','13595','10516', - '90225','58943','39371','94945','28587','96576','57855', - '28488','26105','83933','25858','34322','44438','73171', - '30122','34102','22685','71256','78451','54364','13354', - '45375','40558','56458','28286','45266','47305','69399', - '83921','26233','11101','15371','69913','35942','15882', - '25631','24610','44165','99076','33786','70738','26653', - '14328','72305','62496','22152','10144','64147','48425', - '14663','21076','18799','30450','63089','81019','68893', - '24996','51200','51211','45692','92712','70466','79994', - '22437','25280','38935','71791','73134','56571','14060', - '19505','72425','56575','74351','68786','51650','20004', - '18383','76614','11634','18906','15765','41368','73241', - '76698','78567','97189','28545','76231','75691','22246', - '51061','90578','56691','68014','51103','94167','57047', - '14867','73520','15734','63435','25733','35474','24676', - '94627','53535','17879','15559','53268','59166','11928', - '59402','33282','45721','43933','68101','33515','36634', - '71286','19736','58058','55253','67473','41918','19515', - '36495','19430','22351','77191','91393','49156','50298', - '87501','18652','53179','18767','63193','23968','65164', - '68880','21286','72823','58470','67301','13394','31016', - '70372','67030','40604','24317','45748','39127','26065', - '77721','31029','31880','60576','24671','45549','13376', - '50016','33123','19769','22927','97789','46081','72151', - '15723','46136','51949','68100','96888','64528','14171', - '79777','28709','11489','25103','32213','78668','22245', - '15798','27156','37930','62971','21337','51622','67853', - '10567','38415','15455','58263','42029','60279','37125', - '56240','88190','50308','26859','64457','89091','82136', - '62377','36233','63837','58078','17043','30010','60099', - '28810','98025','29178','87343','73273','30469','64034', - '39516','86057','21309','90257','67875','40162','11356', - '73650','61810','72013','30431','22461','19512','13375', - '55307','30625','83849','68908','26689','96451','38193', - '46820','88885','84935','69035','83144','47537','56616', - '94983','48033','69952','25486','61547','27385','61860', - '58048','56910','16807','17871','35258','31387','35458', - '35576')) - INTERSECT - (SELECT ca_zip - FROM - (SELECT - substr(ca_zip, 1, 5) ca_zip, - count(*) cnt - FROM customer_address, customer - WHERE ca_address_sk = c_current_addr_sk AND - c_preferred_cust_flag = 'Y' - GROUP BY ca_zip - HAVING count(*) > 10) A1) - ) A2 - ) V1 -WHERE ss_store_sk = s_store_sk - AND ss_sold_date_sk = d_date_sk - AND d_qoy = 2 AND d_year = 1998 - AND (substr(s_zip, 1, 2) = substr(V1.ca_zip, 1, 2)) -GROUP BY s_store_name -ORDER BY s_store_name -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q80.sql b/src/test/resources/tpcds/queries/q80.sql deleted file mode 100644 index 433db87d2..000000000 --- a/src/test/resources/tpcds/queries/q80.sql +++ /dev/null @@ -1,94 +0,0 @@ -WITH ssr AS -(SELECT - s_store_id AS store_id, - sum(ss_ext_sales_price) AS sales, - sum(coalesce(sr_return_amt, 0)) AS returns, - sum(ss_net_profit - coalesce(sr_net_loss, 0)) AS profit - FROM store_sales - LEFT OUTER JOIN store_returns ON - (ss_item_sk = sr_item_sk AND - ss_ticket_number = sr_ticket_number) - , - date_dim, store, item, promotion - WHERE ss_sold_date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-23' AS DATE) - AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) - AND ss_store_sk = s_store_sk - AND ss_item_sk = i_item_sk - AND i_current_price > 50 - AND ss_promo_sk = p_promo_sk - AND p_channel_tv = 'N' - GROUP BY s_store_id), - csr AS - (SELECT - cp_catalog_page_id AS catalog_page_id, - sum(cs_ext_sales_price) AS sales, - sum(coalesce(cr_return_amount, 0)) AS returns, - sum(cs_net_profit - coalesce(cr_net_loss, 0)) AS profit - FROM catalog_sales - LEFT OUTER JOIN catalog_returns ON - (cs_item_sk = cr_item_sk AND - cs_order_number = cr_order_number) - , - date_dim, catalog_page, item, promotion - WHERE cs_sold_date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-23' AS DATE) - AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) - AND cs_catalog_page_sk = cp_catalog_page_sk - AND cs_item_sk = i_item_sk - AND i_current_price > 50 - AND cs_promo_sk = p_promo_sk - AND p_channel_tv = 'N' - GROUP BY cp_catalog_page_id), - wsr AS - (SELECT - web_site_id, - sum(ws_ext_sales_price) AS sales, - sum(coalesce(wr_return_amt, 0)) AS returns, - sum(ws_net_profit - coalesce(wr_net_loss, 0)) AS profit - FROM web_sales - LEFT OUTER JOIN web_returns ON - (ws_item_sk = wr_item_sk AND ws_order_number = wr_order_number) - , - date_dim, web_site, item, promotion - WHERE ws_sold_date_sk = d_date_sk - AND d_date BETWEEN cast('2000-08-23' AS DATE) - AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) - AND ws_web_site_sk = web_site_sk - AND ws_item_sk = i_item_sk - AND i_current_price > 50 - AND ws_promo_sk = p_promo_sk - AND p_channel_tv = 'N' - GROUP BY web_site_id) -SELECT - channel, - id, - sum(sales) AS sales, - sum(returns) AS returns, - sum(profit) AS profit -FROM (SELECT - 'store channel' AS channel, - concat('store', store_id) AS id, - sales, - returns, - profit - FROM ssr - UNION ALL - SELECT - 'catalog channel' AS channel, - concat('catalog_page', catalog_page_id) AS id, - sales, - returns, - profit - FROM csr - UNION ALL - SELECT - 'web channel' AS channel, - concat('web_site', web_site_id) AS id, - sales, - returns, - profit - FROM wsr) x -GROUP BY ROLLUP (channel, id) -ORDER BY channel, id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q81.sql b/src/test/resources/tpcds/queries/q81.sql deleted file mode 100644 index 18f0ffa7e..000000000 --- a/src/test/resources/tpcds/queries/q81.sql +++ /dev/null @@ -1,38 +0,0 @@ -WITH customer_total_return AS -(SELECT - cr_returning_customer_sk AS ctr_customer_sk, - ca_state AS ctr_state, - sum(cr_return_amt_inc_tax) AS ctr_total_return - FROM catalog_returns, date_dim, customer_address - WHERE cr_returned_date_sk = d_date_sk - AND d_year = 2000 - AND cr_returning_addr_sk = ca_address_sk - GROUP BY cr_returning_customer_sk, ca_state ) -SELECT - c_customer_id, - c_salutation, - c_first_name, - c_last_name, - ca_street_number, - ca_street_name, - ca_street_type, - ca_suite_number, - ca_city, - ca_county, - ca_state, - ca_zip, - ca_country, - ca_gmt_offset, - ca_location_type, - ctr_total_return -FROM customer_total_return ctr1, customer_address, customer -WHERE ctr1.ctr_total_return > (SELECT avg(ctr_total_return) * 1.2 -FROM customer_total_return ctr2 -WHERE ctr1.ctr_state = ctr2.ctr_state) - AND ca_address_sk = c_current_addr_sk - AND ca_state = 'GA' - AND ctr1.ctr_customer_sk = c_customer_sk -ORDER BY c_customer_id, c_salutation, c_first_name, c_last_name, ca_street_number, ca_street_name - , ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset - , ca_location_type, ctr_total_return -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q82.sql b/src/test/resources/tpcds/queries/q82.sql deleted file mode 100644 index 20942cfeb..000000000 --- a/src/test/resources/tpcds/queries/q82.sql +++ /dev/null @@ -1,15 +0,0 @@ -SELECT - i_item_id, - i_item_desc, - i_current_price -FROM item, inventory, date_dim, store_sales -WHERE i_current_price BETWEEN 62 AND 62 + 30 - AND inv_item_sk = i_item_sk - AND d_date_sk = inv_date_sk - AND d_date BETWEEN cast('2000-05-25' AS DATE) AND (cast('2000-05-25' AS DATE) + INTERVAL 60 days) - AND i_manufact_id IN (129, 270, 821, 423) - AND inv_quantity_on_hand BETWEEN 100 AND 500 - AND ss_item_sk = i_item_sk -GROUP BY i_item_id, i_item_desc, i_current_price -ORDER BY i_item_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q83.sql b/src/test/resources/tpcds/queries/q83.sql deleted file mode 100644 index 53c10c7de..000000000 --- a/src/test/resources/tpcds/queries/q83.sql +++ /dev/null @@ -1,56 +0,0 @@ -WITH sr_items AS -(SELECT - i_item_id item_id, - sum(sr_return_quantity) sr_item_qty - FROM store_returns, item, date_dim - WHERE sr_item_sk = i_item_sk - AND d_date IN (SELECT d_date - FROM date_dim - WHERE d_week_seq IN - (SELECT d_week_seq - FROM date_dim - WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) - AND sr_returned_date_sk = d_date_sk - GROUP BY i_item_id), - cr_items AS - (SELECT - i_item_id item_id, - sum(cr_return_quantity) cr_item_qty - FROM catalog_returns, item, date_dim - WHERE cr_item_sk = i_item_sk - AND d_date IN (SELECT d_date - FROM date_dim - WHERE d_week_seq IN - (SELECT d_week_seq - FROM date_dim - WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) - AND cr_returned_date_sk = d_date_sk - GROUP BY i_item_id), - wr_items AS - (SELECT - i_item_id item_id, - sum(wr_return_quantity) wr_item_qty - FROM web_returns, item, date_dim - WHERE wr_item_sk = i_item_sk AND d_date IN - (SELECT d_date - FROM date_dim - WHERE d_week_seq IN - (SELECT d_week_seq - FROM date_dim - WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) - AND wr_returned_date_sk = d_date_sk - GROUP BY i_item_id) -SELECT - sr_items.item_id, - sr_item_qty, - sr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 sr_dev, - cr_item_qty, - cr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 cr_dev, - wr_item_qty, - wr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 wr_dev, - (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 average -FROM sr_items, cr_items, wr_items -WHERE sr_items.item_id = cr_items.item_id - AND sr_items.item_id = wr_items.item_id -ORDER BY sr_items.item_id, sr_item_qty -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q84.sql b/src/test/resources/tpcds/queries/q84.sql deleted file mode 100644 index a1076b57c..000000000 --- a/src/test/resources/tpcds/queries/q84.sql +++ /dev/null @@ -1,19 +0,0 @@ -SELECT - c_customer_id AS customer_id, - concat(c_last_name, ', ', c_first_name) AS customername -FROM customer - , customer_address - , customer_demographics - , household_demographics - , income_band - , store_returns -WHERE ca_city = 'Edgewood' - AND c_current_addr_sk = ca_address_sk - AND ib_lower_bound >= 38128 - AND ib_upper_bound <= 38128 + 50000 - AND ib_income_band_sk = hd_income_band_sk - AND cd_demo_sk = c_current_cdemo_sk - AND hd_demo_sk = c_current_hdemo_sk - AND sr_cdemo_sk = cd_demo_sk -ORDER BY c_customer_id -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q85.sql b/src/test/resources/tpcds/queries/q85.sql deleted file mode 100644 index cf718b0f8..000000000 --- a/src/test/resources/tpcds/queries/q85.sql +++ /dev/null @@ -1,82 +0,0 @@ -SELECT - substr(r_reason_desc, 1, 20), - avg(ws_quantity), - avg(wr_refunded_cash), - avg(wr_fee) -FROM web_sales, web_returns, web_page, customer_demographics cd1, - customer_demographics cd2, customer_address, date_dim, reason -WHERE ws_web_page_sk = wp_web_page_sk - AND ws_item_sk = wr_item_sk - AND ws_order_number = wr_order_number - AND ws_sold_date_sk = d_date_sk AND d_year = 2000 - AND cd1.cd_demo_sk = wr_refunded_cdemo_sk - AND cd2.cd_demo_sk = wr_returning_cdemo_sk - AND ca_address_sk = wr_refunded_addr_sk - AND r_reason_sk = wr_reason_sk - AND - ( - ( - cd1.cd_marital_status = 'M' - AND - cd1.cd_marital_status = cd2.cd_marital_status - AND - cd1.cd_education_status = 'Advanced Degree' - AND - cd1.cd_education_status = cd2.cd_education_status - AND - ws_sales_price BETWEEN 100.00 AND 150.00 - ) - OR - ( - cd1.cd_marital_status = 'S' - AND - cd1.cd_marital_status = cd2.cd_marital_status - AND - cd1.cd_education_status = 'College' - AND - cd1.cd_education_status = cd2.cd_education_status - AND - ws_sales_price BETWEEN 50.00 AND 100.00 - ) - OR - ( - cd1.cd_marital_status = 'W' - AND - cd1.cd_marital_status = cd2.cd_marital_status - AND - cd1.cd_education_status = '2 yr Degree' - AND - cd1.cd_education_status = cd2.cd_education_status - AND - ws_sales_price BETWEEN 150.00 AND 200.00 - ) - ) - AND - ( - ( - ca_country = 'United States' - AND - ca_state IN ('IN', 'OH', 'NJ') - AND ws_net_profit BETWEEN 100 AND 200 - ) - OR - ( - ca_country = 'United States' - AND - ca_state IN ('WI', 'CT', 'KY') - AND ws_net_profit BETWEEN 150 AND 300 - ) - OR - ( - ca_country = 'United States' - AND - ca_state IN ('LA', 'IA', 'AR') - AND ws_net_profit BETWEEN 50 AND 250 - ) - ) -GROUP BY r_reason_desc -ORDER BY substr(r_reason_desc, 1, 20) - , avg(ws_quantity) - , avg(wr_refunded_cash) - , avg(wr_fee) -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q86.sql b/src/test/resources/tpcds/queries/q86.sql deleted file mode 100644 index 789a4abf7..000000000 --- a/src/test/resources/tpcds/queries/q86.sql +++ /dev/null @@ -1,24 +0,0 @@ -SELECT - sum(ws_net_paid) AS total_sum, - i_category, - i_class, - grouping(i_category) + grouping(i_class) AS lochierarchy, - rank() - OVER ( - PARTITION BY grouping(i_category) + grouping(i_class), - CASE WHEN grouping(i_class) = 0 - THEN i_category END - ORDER BY sum(ws_net_paid) DESC) AS rank_within_parent -FROM - web_sales, date_dim d1, item -WHERE - d1.d_month_seq BETWEEN 1200 AND 1200 + 11 - AND d1.d_date_sk = ws_sold_date_sk - AND i_item_sk = ws_item_sk -GROUP BY ROLLUP (i_category, i_class) -ORDER BY - lochierarchy DESC, - CASE WHEN lochierarchy = 0 - THEN i_category END, - rank_within_parent -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q87.sql b/src/test/resources/tpcds/queries/q87.sql deleted file mode 100644 index 4aaa9f39d..000000000 --- a/src/test/resources/tpcds/queries/q87.sql +++ /dev/null @@ -1,28 +0,0 @@ -SELECT count(*) -FROM ((SELECT DISTINCT - c_last_name, - c_first_name, - d_date -FROM store_sales, date_dim, customer -WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk - AND store_sales.ss_customer_sk = customer.c_customer_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11) - EXCEPT - (SELECT DISTINCT - c_last_name, - c_first_name, - d_date - FROM catalog_sales, date_dim, customer - WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk - AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11) - EXCEPT - (SELECT DISTINCT - c_last_name, - c_first_name, - d_date - FROM web_sales, date_dim, customer - WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk - AND web_sales.ws_bill_customer_sk = customer.c_customer_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11) - ) cool_cust diff --git a/src/test/resources/tpcds/queries/q88.sql b/src/test/resources/tpcds/queries/q88.sql deleted file mode 100644 index 25bcd90f4..000000000 --- a/src/test/resources/tpcds/queries/q88.sql +++ /dev/null @@ -1,122 +0,0 @@ -SELECT * -FROM - (SELECT count(*) h8_30_to_9 - FROM store_sales, household_demographics, time_dim, store - WHERE ss_sold_time_sk = time_dim.t_time_sk - AND ss_hdemo_sk = household_demographics.hd_demo_sk - AND ss_store_sk = s_store_sk - AND time_dim.t_hour = 8 - AND time_dim.t_minute >= 30 - AND ( - (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) - OR - (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) - OR - (household_demographics.hd_dep_count = 0 AND - household_demographics.hd_vehicle_count <= 0 + 2)) - AND store.s_store_name = 'ese') s1, - (SELECT count(*) h9_to_9_30 - FROM store_sales, household_demographics, time_dim, store - WHERE ss_sold_time_sk = time_dim.t_time_sk - AND ss_hdemo_sk = household_demographics.hd_demo_sk - AND ss_store_sk = s_store_sk - AND time_dim.t_hour = 9 - AND time_dim.t_minute < 30 - AND ( - (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) - OR - (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) - OR - (household_demographics.hd_dep_count = 0 AND - household_demographics.hd_vehicle_count <= 0 + 2)) - AND store.s_store_name = 'ese') s2, - (SELECT count(*) h9_30_to_10 - FROM store_sales, household_demographics, time_dim, store - WHERE ss_sold_time_sk = time_dim.t_time_sk - AND ss_hdemo_sk = household_demographics.hd_demo_sk - AND ss_store_sk = s_store_sk - AND time_dim.t_hour = 9 - AND time_dim.t_minute >= 30 - AND ( - (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) - OR - (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) - OR - (household_demographics.hd_dep_count = 0 AND - household_demographics.hd_vehicle_count <= 0 + 2)) - AND store.s_store_name = 'ese') s3, - (SELECT count(*) h10_to_10_30 - FROM store_sales, household_demographics, time_dim, store - WHERE ss_sold_time_sk = time_dim.t_time_sk - AND ss_hdemo_sk = household_demographics.hd_demo_sk - AND ss_store_sk = s_store_sk - AND time_dim.t_hour = 10 - AND time_dim.t_minute < 30 - AND ( - (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) - OR - (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) - OR - (household_demographics.hd_dep_count = 0 AND - household_demographics.hd_vehicle_count <= 0 + 2)) - AND store.s_store_name = 'ese') s4, - (SELECT count(*) h10_30_to_11 - FROM store_sales, household_demographics, time_dim, store - WHERE ss_sold_time_sk = time_dim.t_time_sk - AND ss_hdemo_sk = household_demographics.hd_demo_sk - AND ss_store_sk = s_store_sk - AND time_dim.t_hour = 10 - AND time_dim.t_minute >= 30 - AND ( - (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) - OR - (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) - OR - (household_demographics.hd_dep_count = 0 AND - household_demographics.hd_vehicle_count <= 0 + 2)) - AND store.s_store_name = 'ese') s5, - (SELECT count(*) h11_to_11_30 - FROM store_sales, household_demographics, time_dim, store - WHERE ss_sold_time_sk = time_dim.t_time_sk - AND ss_hdemo_sk = household_demographics.hd_demo_sk - AND ss_store_sk = s_store_sk - AND time_dim.t_hour = 11 - AND time_dim.t_minute < 30 - AND ( - (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) - OR - (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) - OR - (household_demographics.hd_dep_count = 0 AND - household_demographics.hd_vehicle_count <= 0 + 2)) - AND store.s_store_name = 'ese') s6, - (SELECT count(*) h11_30_to_12 - FROM store_sales, household_demographics, time_dim, store - WHERE ss_sold_time_sk = time_dim.t_time_sk - AND ss_hdemo_sk = household_demographics.hd_demo_sk - AND ss_store_sk = s_store_sk - AND time_dim.t_hour = 11 - AND time_dim.t_minute >= 30 - AND ( - (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) - OR - (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) - OR - (household_demographics.hd_dep_count = 0 AND - household_demographics.hd_vehicle_count <= 0 + 2)) - AND store.s_store_name = 'ese') s7, - (SELECT count(*) h12_to_12_30 - FROM store_sales, household_demographics, time_dim, store - WHERE ss_sold_time_sk = time_dim.t_time_sk - AND ss_hdemo_sk = household_demographics.hd_demo_sk - AND ss_store_sk = s_store_sk - AND time_dim.t_hour = 12 - AND time_dim.t_minute < 30 - AND ( - (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) - OR - (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) - OR - (household_demographics.hd_dep_count = 0 AND - household_demographics.hd_vehicle_count <= 0 + 2)) - AND store.s_store_name = 'ese') s8 diff --git a/src/test/resources/tpcds/queries/q89.sql b/src/test/resources/tpcds/queries/q89.sql deleted file mode 100644 index 75408cb03..000000000 --- a/src/test/resources/tpcds/queries/q89.sql +++ /dev/null @@ -1,30 +0,0 @@ -SELECT * -FROM ( - SELECT - i_category, - i_class, - i_brand, - s_store_name, - s_company_name, - d_moy, - sum(ss_sales_price) sum_sales, - avg(sum(ss_sales_price)) - OVER - (PARTITION BY i_category, i_brand, s_store_name, s_company_name) - avg_monthly_sales - FROM item, store_sales, date_dim, store - WHERE ss_item_sk = i_item_sk AND - ss_sold_date_sk = d_date_sk AND - ss_store_sk = s_store_sk AND - d_year IN (1999) AND - ((i_category IN ('Books', 'Electronics', 'Sports') AND - i_class IN ('computers', 'stereo', 'football')) - OR (i_category IN ('Men', 'Jewelry', 'Women') AND - i_class IN ('shirts', 'birdal', 'dresses'))) - GROUP BY i_category, i_class, i_brand, - s_store_name, s_company_name, d_moy) tmp1 -WHERE CASE WHEN (avg_monthly_sales <> 0) - THEN (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) - ELSE NULL END > 0.1 -ORDER BY sum_sales - avg_monthly_sales, s_store_name -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q9.sql b/src/test/resources/tpcds/queries/q9.sql deleted file mode 100644 index de3db9d98..000000000 --- a/src/test/resources/tpcds/queries/q9.sql +++ /dev/null @@ -1,48 +0,0 @@ -SELECT - CASE WHEN (SELECT count(*) - FROM store_sales - WHERE ss_quantity BETWEEN 1 AND 20) > 62316685 - THEN (SELECT avg(ss_ext_discount_amt) - FROM store_sales - WHERE ss_quantity BETWEEN 1 AND 20) - ELSE (SELECT avg(ss_net_paid) - FROM store_sales - WHERE ss_quantity BETWEEN 1 AND 20) END bucket1, - CASE WHEN (SELECT count(*) - FROM store_sales - WHERE ss_quantity BETWEEN 21 AND 40) > 19045798 - THEN (SELECT avg(ss_ext_discount_amt) - FROM store_sales - WHERE ss_quantity BETWEEN 21 AND 40) - ELSE (SELECT avg(ss_net_paid) - FROM store_sales - WHERE ss_quantity BETWEEN 21 AND 40) END bucket2, - CASE WHEN (SELECT count(*) - FROM store_sales - WHERE ss_quantity BETWEEN 41 AND 60) > 365541424 - THEN (SELECT avg(ss_ext_discount_amt) - FROM store_sales - WHERE ss_quantity BETWEEN 41 AND 60) - ELSE (SELECT avg(ss_net_paid) - FROM store_sales - WHERE ss_quantity BETWEEN 41 AND 60) END bucket3, - CASE WHEN (SELECT count(*) - FROM store_sales - WHERE ss_quantity BETWEEN 61 AND 80) > 216357808 - THEN (SELECT avg(ss_ext_discount_amt) - FROM store_sales - WHERE ss_quantity BETWEEN 61 AND 80) - ELSE (SELECT avg(ss_net_paid) - FROM store_sales - WHERE ss_quantity BETWEEN 61 AND 80) END bucket4, - CASE WHEN (SELECT count(*) - FROM store_sales - WHERE ss_quantity BETWEEN 81 AND 100) > 184483884 - THEN (SELECT avg(ss_ext_discount_amt) - FROM store_sales - WHERE ss_quantity BETWEEN 81 AND 100) - ELSE (SELECT avg(ss_net_paid) - FROM store_sales - WHERE ss_quantity BETWEEN 81 AND 100) END bucket5 -FROM reason -WHERE r_reason_sk = 1 diff --git a/src/test/resources/tpcds/queries/q90.sql b/src/test/resources/tpcds/queries/q90.sql deleted file mode 100644 index 85e35bf8b..000000000 --- a/src/test/resources/tpcds/queries/q90.sql +++ /dev/null @@ -1,19 +0,0 @@ -SELECT cast(amc AS DECIMAL(15, 4)) / cast(pmc AS DECIMAL(15, 4)) am_pm_ratio -FROM (SELECT count(*) amc -FROM web_sales, household_demographics, time_dim, web_page -WHERE ws_sold_time_sk = time_dim.t_time_sk - AND ws_ship_hdemo_sk = household_demographics.hd_demo_sk - AND ws_web_page_sk = web_page.wp_web_page_sk - AND time_dim.t_hour BETWEEN 8 AND 8 + 1 - AND household_demographics.hd_dep_count = 6 - AND web_page.wp_char_count BETWEEN 5000 AND 5200) at, - (SELECT count(*) pmc - FROM web_sales, household_demographics, time_dim, web_page - WHERE ws_sold_time_sk = time_dim.t_time_sk - AND ws_ship_hdemo_sk = household_demographics.hd_demo_sk - AND ws_web_page_sk = web_page.wp_web_page_sk - AND time_dim.t_hour BETWEEN 19 AND 19 + 1 - AND household_demographics.hd_dep_count = 6 - AND web_page.wp_char_count BETWEEN 5000 AND 5200) pt -ORDER BY am_pm_ratio -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q91.sql b/src/test/resources/tpcds/queries/q91.sql deleted file mode 100644 index 9ca7ce00a..000000000 --- a/src/test/resources/tpcds/queries/q91.sql +++ /dev/null @@ -1,23 +0,0 @@ -SELECT - cc_call_center_id Call_Center, - cc_name Call_Center_Name, - cc_manager Manager, - sum(cr_net_loss) Returns_Loss -FROM - call_center, catalog_returns, date_dim, customer, customer_address, - customer_demographics, household_demographics -WHERE - cr_call_center_sk = cc_call_center_sk - AND cr_returned_date_sk = d_date_sk - AND cr_returning_customer_sk = c_customer_sk - AND cd_demo_sk = c_current_cdemo_sk - AND hd_demo_sk = c_current_hdemo_sk - AND ca_address_sk = c_current_addr_sk - AND d_year = 1998 - AND d_moy = 11 - AND ((cd_marital_status = 'M' AND cd_education_status = 'Unknown') - OR (cd_marital_status = 'W' AND cd_education_status = 'Advanced Degree')) - AND hd_buy_potential LIKE 'Unknown%' - AND ca_gmt_offset = -7 -GROUP BY cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status -ORDER BY sum(cr_net_loss) DESC diff --git a/src/test/resources/tpcds/queries/q92.sql b/src/test/resources/tpcds/queries/q92.sql deleted file mode 100644 index 99129c3bd..000000000 --- a/src/test/resources/tpcds/queries/q92.sql +++ /dev/null @@ -1,16 +0,0 @@ -SELECT sum(ws_ext_discount_amt) AS `Excess Discount Amount ` -FROM web_sales, item, date_dim -WHERE i_manufact_id = 350 - AND i_item_sk = ws_item_sk - AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) - AND d_date_sk = ws_sold_date_sk - AND ws_ext_discount_amt > - ( - SELECT 1.3 * avg(ws_ext_discount_amt) - FROM web_sales, date_dim - WHERE ws_item_sk = i_item_sk - AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) - AND d_date_sk = ws_sold_date_sk - ) -ORDER BY sum(ws_ext_discount_amt) -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q93.sql b/src/test/resources/tpcds/queries/q93.sql deleted file mode 100644 index 222dc31c1..000000000 --- a/src/test/resources/tpcds/queries/q93.sql +++ /dev/null @@ -1,19 +0,0 @@ -SELECT - ss_customer_sk, - sum(act_sales) sumsales -FROM (SELECT - ss_item_sk, - ss_ticket_number, - ss_customer_sk, - CASE WHEN sr_return_quantity IS NOT NULL - THEN (ss_quantity - sr_return_quantity) * ss_sales_price - ELSE (ss_quantity * ss_sales_price) END act_sales -FROM store_sales - LEFT OUTER JOIN store_returns - ON (sr_item_sk = ss_item_sk AND sr_ticket_number = ss_ticket_number) - , - reason -WHERE sr_reason_sk = r_reason_sk AND r_reason_desc = 'reason 28') t -GROUP BY ss_customer_sk -ORDER BY sumsales, ss_customer_sk -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q94.sql b/src/test/resources/tpcds/queries/q94.sql deleted file mode 100644 index d6de3d75b..000000000 --- a/src/test/resources/tpcds/queries/q94.sql +++ /dev/null @@ -1,23 +0,0 @@ -SELECT - count(DISTINCT ws_order_number) AS `order count `, - sum(ws_ext_ship_cost) AS `total shipping cost `, - sum(ws_net_profit) AS `total net profit ` -FROM - web_sales ws1, date_dim, customer_address, web_site -WHERE - d_date BETWEEN '1999-02-01' AND - (CAST('1999-02-01' AS DATE) + INTERVAL 60 days) - AND ws1.ws_ship_date_sk = d_date_sk - AND ws1.ws_ship_addr_sk = ca_address_sk - AND ca_state = 'IL' - AND ws1.ws_web_site_sk = web_site_sk - AND web_company_name = 'pri' - AND EXISTS(SELECT * - FROM web_sales ws2 - WHERE ws1.ws_order_number = ws2.ws_order_number - AND ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) - AND NOT EXISTS(SELECT * - FROM web_returns wr1 - WHERE ws1.ws_order_number = wr1.wr_order_number) -ORDER BY count(DISTINCT ws_order_number) -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q95.sql b/src/test/resources/tpcds/queries/q95.sql deleted file mode 100644 index df71f00bd..000000000 --- a/src/test/resources/tpcds/queries/q95.sql +++ /dev/null @@ -1,29 +0,0 @@ -WITH ws_wh AS -(SELECT - ws1.ws_order_number, - ws1.ws_warehouse_sk wh1, - ws2.ws_warehouse_sk wh2 - FROM web_sales ws1, web_sales ws2 - WHERE ws1.ws_order_number = ws2.ws_order_number - AND ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) -SELECT - count(DISTINCT ws_order_number) AS `order count `, - sum(ws_ext_ship_cost) AS `total shipping cost `, - sum(ws_net_profit) AS `total net profit ` -FROM - web_sales ws1, date_dim, customer_address, web_site -WHERE - d_date BETWEEN '1999-02-01' AND - (CAST('1999-02-01' AS DATE) + INTERVAL 60 DAY) - AND ws1.ws_ship_date_sk = d_date_sk - AND ws1.ws_ship_addr_sk = ca_address_sk - AND ca_state = 'IL' - AND ws1.ws_web_site_sk = web_site_sk - AND web_company_name = 'pri' - AND ws1.ws_order_number IN (SELECT ws_order_number - FROM ws_wh) - AND ws1.ws_order_number IN (SELECT wr_order_number - FROM web_returns, ws_wh - WHERE wr_order_number = ws_wh.ws_order_number) -ORDER BY count(DISTINCT ws_order_number) -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q96.sql b/src/test/resources/tpcds/queries/q96.sql deleted file mode 100644 index 7ab17e7bc..000000000 --- a/src/test/resources/tpcds/queries/q96.sql +++ /dev/null @@ -1,11 +0,0 @@ -SELECT count(*) -FROM store_sales, household_demographics, time_dim, store -WHERE ss_sold_time_sk = time_dim.t_time_sk - AND ss_hdemo_sk = household_demographics.hd_demo_sk - AND ss_store_sk = s_store_sk - AND time_dim.t_hour = 20 - AND time_dim.t_minute >= 30 - AND household_demographics.hd_dep_count = 7 - AND store.s_store_name = 'ese' -ORDER BY count(*) -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q97.sql b/src/test/resources/tpcds/queries/q97.sql deleted file mode 100644 index e7e0b1a05..000000000 --- a/src/test/resources/tpcds/queries/q97.sql +++ /dev/null @@ -1,30 +0,0 @@ -WITH ssci AS ( - SELECT - ss_customer_sk customer_sk, - ss_item_sk item_sk - FROM store_sales, date_dim - WHERE ss_sold_date_sk = d_date_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11 - GROUP BY ss_customer_sk, ss_item_sk), - csci AS ( - SELECT - cs_bill_customer_sk customer_sk, - cs_item_sk item_sk - FROM catalog_sales, date_dim - WHERE cs_sold_date_sk = d_date_sk - AND d_month_seq BETWEEN 1200 AND 1200 + 11 - GROUP BY cs_bill_customer_sk, cs_item_sk) -SELECT - sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NULL - THEN 1 - ELSE 0 END) store_only, - sum(CASE WHEN ssci.customer_sk IS NULL AND csci.customer_sk IS NOT NULL - THEN 1 - ELSE 0 END) catalog_only, - sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NOT NULL - THEN 1 - ELSE 0 END) store_and_catalog -FROM ssci - FULL OUTER JOIN csci ON (ssci.customer_sk = csci.customer_sk - AND ssci.item_sk = csci.item_sk) -LIMIT 100 diff --git a/src/test/resources/tpcds/queries/q98.sql b/src/test/resources/tpcds/queries/q98.sql deleted file mode 100644 index bb10d4bf8..000000000 --- a/src/test/resources/tpcds/queries/q98.sql +++ /dev/null @@ -1,21 +0,0 @@ -SELECT - i_item_desc, - i_category, - i_class, - i_current_price, - sum(ss_ext_sales_price) AS itemrevenue, - sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) - OVER - (PARTITION BY i_class) AS revenueratio -FROM - store_sales, item, date_dim -WHERE - ss_item_sk = i_item_sk - AND i_category IN ('Sports', 'Books', 'Home') - AND ss_sold_date_sk = d_date_sk - AND d_date BETWEEN cast('1999-02-22' AS DATE) - AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) -GROUP BY - i_item_id, i_item_desc, i_category, i_class, i_current_price -ORDER BY - i_category, i_class, i_item_id, i_item_desc, revenueratio diff --git a/src/test/resources/tpcds/queries/q99.sql b/src/test/resources/tpcds/queries/q99.sql deleted file mode 100644 index f1a3d4d2b..000000000 --- a/src/test/resources/tpcds/queries/q99.sql +++ /dev/null @@ -1,34 +0,0 @@ -SELECT - substr(w_warehouse_name, 1, 20), - sm_type, - cc_name, - sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk <= 30) - THEN 1 - ELSE 0 END) AS `30 days `, - sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 30) AND - (cs_ship_date_sk - cs_sold_date_sk <= 60) - THEN 1 - ELSE 0 END) AS `31 - 60 days `, - sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 60) AND - (cs_ship_date_sk - cs_sold_date_sk <= 90) - THEN 1 - ELSE 0 END) AS `61 - 90 days `, - sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 90) AND - (cs_ship_date_sk - cs_sold_date_sk <= 120) - THEN 1 - ELSE 0 END) AS `91 - 120 days `, - sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 120) - THEN 1 - ELSE 0 END) AS `>120 days ` -FROM - catalog_sales, warehouse, ship_mode, call_center, date_dim -WHERE - d_month_seq BETWEEN 1200 AND 1200 + 11 - AND cs_ship_date_sk = d_date_sk - AND cs_warehouse_sk = w_warehouse_sk - AND cs_ship_mode_sk = sm_ship_mode_sk - AND cs_call_center_sk = cc_call_center_sk -GROUP BY - substr(w_warehouse_name, 1, 20), sm_type, cc_name -ORDER BY substr(w_warehouse_name, 1, 20), sm_type, cc_name -LIMIT 100 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q1/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q1/explain.txt deleted file mode 100644 index 7899a44d0..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q1/explain.txt +++ /dev/null @@ -1,43 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#1 ASC NULLS FIRST], output=[c_customer_id#1]) -+- *(9) Project [c_customer_id#1] - +- *(9) BroadcastHashJoin [ctr_customer_sk#2], [cast(c_customer_sk#3 as bigint)], Inner, BuildRight - :- *(9) Project [ctr_customer_sk#2] - : +- *(9) BroadcastHashJoin [ctr_store_sk#4], [cast(s_store_sk#5 as bigint)], Inner, BuildRight - : :- *(9) Project [ctr_customer_sk#2, ctr_store_sk#4] - : : +- *(9) BroadcastHashJoin [ctr_store_sk#4], [ctr_store_sk#4#6], Inner, BuildRight, (cast(ctr_total_return#7 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#8) - : : :- *(9) Filter isnotnull(ctr_total_return#7) - : : : +- *(9) HashAggregate(keys=[sr_customer_sk#9, sr_store_sk#10], functions=[sum(UnscaledValue(sr_return_amt#11))]) - : : : +- Exchange hashpartitioning(sr_customer_sk#9, sr_store_sk#10, 5) - : : : +- *(2) HashAggregate(keys=[sr_customer_sk#9, sr_store_sk#10], functions=[partial_sum(UnscaledValue(sr_return_amt#11))]) - : : : +- *(2) Project [sr_customer_sk#9, sr_store_sk#10, sr_return_amt#11] - : : : +- *(2) BroadcastHashJoin [sr_returned_date_sk#12], [cast(d_date_sk#13 as bigint)], Inner, BuildRight - : : : :- *(2) Project [sr_returned_date_sk#12, sr_customer_sk#9, sr_store_sk#10, sr_return_amt#11] - : : : : +- *(2) Filter ((isnotnull(sr_returned_date_sk#12) && isnotnull(sr_store_sk#10)) && isnotnull(sr_customer_sk#9)) - : : : : +- *(2) FileScan parquet default.store_returns[sr_returned_date_sk#12,sr_customer_sk#9,sr_store_sk#10,sr_return_amt#11] Batched: true, Format: Parquet, Location [not included in comparison]/{warehouse_dir}/store_returns], PartitionFilters: [], PushedFilters: [IsNotNull(sr_returned_date_sk), IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)], ReadSchema: struct - : : +- BroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true])) - : : +- *(6) Filter isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#8) - : : +- *(6) HashAggregate(keys=[ctr_store_sk#4], functions=[avg(ctr_total_return#7)]) - : : +- Exchange hashpartitioning(ctr_store_sk#4, 5) - : : +- *(5) HashAggregate(keys=[ctr_store_sk#4], functions=[partial_avg(ctr_total_return#7)]) - : : +- *(5) HashAggregate(keys=[sr_customer_sk#9, sr_store_sk#10], functions=[sum(UnscaledValue(sr_return_amt#11))]) - : : +- Exchange hashpartitioning(sr_customer_sk#9, sr_store_sk#10, 5) - : : +- *(4) HashAggregate(keys=[sr_customer_sk#9, sr_store_sk#10], functions=[partial_sum(UnscaledValue(sr_return_amt#11))]) - : : +- *(4) Project [sr_customer_sk#9, sr_store_sk#10, sr_return_amt#11] - : : +- *(4) BroadcastHashJoin [sr_returned_date_sk#12], [cast(d_date_sk#13 as bigint)], Inner, BuildRight - : : :- *(4) Project [sr_returned_date_sk#12, sr_customer_sk#9, sr_store_sk#10, sr_return_amt#11] - : : : +- *(4) Filter (isnotnull(sr_returned_date_sk#12) && isnotnull(sr_store_sk#10)) - : : : +- *(4) FileScan parquet default.store_returns[sr_returned_date_sk#12,sr_customer_sk#9,sr_store_sk#10,sr_return_amt#11] Batched: true, Format: Parquet, Location [not included in comparison]/{warehouse_dir}/store_returns], PartitionFilters: [], PushedFilters: [IsNotNull(sr_returned_date_sk), IsNotNull(sr_store_sk)], ReadSchema: struct - +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))) - +- *(8) Project [c_customer_sk#3, c_customer_id#1] - +- *(8) Filter isnotnull(c_customer_sk#3) - +- *(8) FileScan parquet default.customer[c_customer_sk#3,c_customer_id#1] Batched: true, Format: Parquet, Location [not included in comparison]/{warehouse_dir}/customer], PartitionFilters: [], PushedFilters: [IsNotNull(c_customer_sk)], ReadSchema: struct \ No newline at end of file diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q1/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q1/simplified.txt deleted file mode 100644 index 4f838850c..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q1/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -TakeOrderedAndProject [c_customer_id] - WholeStageCodegen - Project [c_customer_id] - BroadcastHashJoin [c_customer_sk,ctr_customer_sk] - Project [ctr_customer_sk] - BroadcastHashJoin [ctr_store_sk,s_store_sk] - Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_store_sk,ctr_store_skL,ctr_total_return] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sum,sum(UnscaledValue(sr_return_amt))] - InputAdapter - Exchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen - HashAggregate [sr_customer_sk,sr_return_amt,sr_store_sk,sum,sum] [sum,sum] - Project [sr_customer_sk,sr_return_amt,sr_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Project [sr_customer_sk,sr_return_amt,sr_returned_date_sk,sr_store_sk] - Filter [sr_customer_sk,sr_returned_date_sk,sr_store_sk] - Scan parquet default.store_returns [sr_customer_sk,sr_return_amt,sr_returned_date_sk,sr_store_sk] [sr_customer_sk,sr_return_amt,sr_returned_date_sk,sr_store_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen - Project [d_date_sk] - Filter [d_date_sk,d_year] - Scan parquet default.date_dim [d_date_sk,d_year] [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen - Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [avg(ctr_total_return),count,ctr_store_sk,sum] [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),avg(ctr_total_return),count,ctr_store_skL,sum] - InputAdapter - Exchange [ctr_store_sk] #4 - WholeStageCodegen - HashAggregate [count,count,ctr_store_sk,ctr_total_return,sum,sum] [count,count,sum,sum] - HashAggregate [sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))] [ctr_store_sk,ctr_total_return,sum,sum(UnscaledValue(sr_return_amt))] - InputAdapter - Exchange [sr_customer_sk,sr_store_sk] #5 - WholeStageCodegen - HashAggregate [sr_customer_sk,sr_return_amt,sr_store_sk,sum,sum] [sum,sum] - Project [sr_customer_sk,sr_return_amt,sr_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Project [sr_customer_sk,sr_return_amt,sr_returned_date_sk,sr_store_sk] - Filter [sr_returned_date_sk,sr_store_sk] - Scan parquet default.store_returns [sr_customer_sk,sr_return_amt,sr_returned_date_sk,sr_store_sk] [sr_customer_sk,sr_return_amt,sr_returned_date_sk,sr_store_sk] - InputAdapter - ReusedExchange [d_date_sk] [d_date_sk] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen - Project [s_store_sk] - Filter [s_state,s_store_sk] - Scan parquet default.store [s_state,s_store_sk] [s_state,s_store_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen - Project [c_customer_id,c_customer_sk] - Filter [c_customer_sk] - Scan parquet default.customer [c_customer_id,c_customer_sk] [c_customer_id,c_customer_sk] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q10/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q10/explain.txt deleted file mode 100644 index 509ce3ac4..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q10/explain.txt +++ /dev/null @@ -1,279 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (50) -+- * HashAggregate (49) - +- Exchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * Filter (32) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (31) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (23) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (19) - : : +- BroadcastExchange (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Filter (26) - : : : +- * ColumnarToRow (25) - : : : +- Scan parquet default.catalog_sales (24) - : : +- ReusedExchange (27) - : +- BroadcastExchange (38) - : +- * Project (37) - : +- * Filter (36) - : +- * ColumnarToRow (35) - : +- Scan parquet default.customer_address (34) - +- BroadcastExchange (44) - +- * Filter (43) - +- * ColumnarToRow (42) - +- Scan parquet default.customer_demographics (41) - - -(1) Scan parquet default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(3) Filter [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(4) Scan parquet default.store_sales -Output [2]: [ss_sold_date_sk#6, ss_customer_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_sold_date_sk#6, ss_customer_sk#7] - -(6) Filter [codegen id : 2] -Input [2]: [ss_sold_date_sk#6, ss_customer_sk#7] -Condition : isnotnull(ss_sold_date_sk#6) - -(7) Scan parquet default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_moy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#8, d_year#9, d_moy#10] - -(9) Filter [codegen id : 1] -Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -Condition : (((((isnotnull(d_year#9) AND isnotnull(d_moy#10)) AND (d_year#9 = 2002)) AND (d_moy#10 >= 1)) AND (d_moy#10 <= 4)) AND isnotnull(d_date_sk#8)) - -(10) Project [codegen id : 1] -Output [1]: [d_date_sk#8] -Input [3]: [d_date_sk#8, d_year#9, d_moy#10] - -(11) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(13) Project [codegen id : 2] -Output [1]: [ss_customer_sk#7] -Input [3]: [ss_sold_date_sk#6, ss_customer_sk#7, d_date_sk#8] - -(14) BroadcastExchange -Input [1]: [ss_customer_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(15) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#7] -Join condition: None - -(16) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#13, ws_bill_customer_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 4] -Input [2]: [ws_sold_date_sk#13, ws_bill_customer_sk#14] - -(18) Filter [codegen id : 4] -Input [2]: [ws_sold_date_sk#13, ws_bill_customer_sk#14] -Condition : isnotnull(ws_sold_date_sk#13) - -(19) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#8] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(21) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_sold_date_sk#13, ws_bill_customer_sk#14, d_date_sk#8] - -(22) BroadcastExchange -Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#14] -Join condition: None - -(24) Scan parquet default.catalog_sales -Output [2]: [cs_sold_date_sk#16, cs_ship_customer_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 6] -Input [2]: [cs_sold_date_sk#16, cs_ship_customer_sk#17] - -(26) Filter [codegen id : 6] -Input [2]: [cs_sold_date_sk#16, cs_ship_customer_sk#17] -Condition : isnotnull(cs_sold_date_sk#16) - -(27) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#8] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(29) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_sold_date_sk#16, cs_ship_customer_sk#17, d_date_sk#8] - -(30) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] - -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] -Join condition: None - -(32) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(34) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_county#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_county#20] - -(36) Filter [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_county#20] -Condition : (ca_county#20 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#19)) - -(37) Project [codegen id : 7] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_county#20] - -(38) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#19] -Join condition: None - -(40) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#19] - -(41) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(42) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(43) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) - -(44) BroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(45) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#22] -Join condition: None - -(46) Project [codegen id : 9] -Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(47) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#32] -Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] - -(48) Exchange -Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), true, [id=#34] - -(49) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#35 AS cnt1#36, cd_purchase_estimate#26, count(1)#35 AS cnt2#37, cd_credit_rating#27, count(1)#35 AS cnt3#38, cd_dep_count#28, count(1)#35 AS cnt4#39, cd_dep_employed_count#29, count(1)#35 AS cnt5#40, cd_dep_college_count#30, count(1)#35 AS cnt6#41] - -(50) TakeOrderedAndProject -Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q10/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q10/simplified.txt deleted file mode 100644 index 2eb1bd3c5..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q10/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - InputAdapter - Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - Project [ca_address_sk] - Filter [ca_county,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q11/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q11/explain.txt deleted file mode 100644 index 246a5d5b0..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q11/explain.txt +++ /dev/null @@ -1,415 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (73) -+- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (57) - : +- * BroadcastHashJoin Inner BuildRight (56) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Filter (19) - : : : +- * HashAggregate (18) - : : : +- Exchange (17) - : : : +- * HashAggregate (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_sales (4) - : : : +- BroadcastExchange (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (35) - : : +- * HashAggregate (34) - : : +- Exchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * Filter (22) - : : : : +- * ColumnarToRow (21) - : : : : +- Scan parquet default.customer (20) - : : : +- ReusedExchange (23) - : : +- BroadcastExchange (29) - : : +- * Filter (28) - : : +- * ColumnarToRow (27) - : : +- Scan parquet default.date_dim (26) - : +- BroadcastExchange (55) - : +- * Project (54) - : +- * Filter (53) - : +- * HashAggregate (52) - : +- Exchange (51) - : +- * HashAggregate (50) - : +- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (46) - : : +- * BroadcastHashJoin Inner BuildRight (45) - : : :- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.customer (38) - : : +- BroadcastExchange (44) - : : +- * Filter (43) - : : +- * ColumnarToRow (42) - : : +- Scan parquet default.web_sales (41) - : +- ReusedExchange (47) - +- BroadcastExchange (70) - +- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Filter (60) - : : +- * ColumnarToRow (59) - : : +- Scan parquet default.customer (58) - : +- ReusedExchange (61) - +- ReusedExchange (64) - - -(1) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] - -(3) Filter [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(4) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_list_price#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_list_price#12] - -(6) Filter [codegen id : 1] -Input [4]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_list_price#12] -Condition : (isnotnull(ss_customer_sk#10) AND isnotnull(ss_sold_date_sk#9)) - -(7) BroadcastExchange -Input [4]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_list_price#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#13] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#10] -Join condition: None - -(9) Project [codegen id : 3] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_ext_discount_amt#11, ss_ext_list_price#12] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_list_price#12] - -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#14, d_year#15] - -(12) Filter [codegen id : 2] -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) - -(13) BroadcastExchange -Input [2]: [d_date_sk#14, d_year#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] - -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#14] -Join condition: None - -(15) Project [codegen id : 3] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#11, ss_ext_list_price#12, d_year#15] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_ext_discount_amt#11, ss_ext_list_price#12, d_date_sk#14, d_year#15] - -(16) HashAggregate [codegen id : 3] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#11, ss_ext_list_price#12, d_year#15] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#12 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#17] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#18] - -(17) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#18] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), true, [id=#19] - -(18) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#12 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#12 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(8,2)))), DecimalType(8,2), true)))#20] -Results [2]: [c_customer_id#2 AS customer_id#21, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#12 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(8,2)))), DecimalType(8,2), true)))#20,18,2) AS year_total#22] - -(19) Filter [codegen id : 16] -Input [2]: [customer_id#21, year_total#22] -Condition : (isnotnull(year_total#22) AND (year_total#22 > 0.00)) - -(20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] - -(22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(23) ReusedExchange [Reuses operator id: 7] -Output [4]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_list_price#12] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#10] -Join condition: None - -(25) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_ext_discount_amt#11, ss_ext_list_price#12] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_list_price#12] - -(26) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#14, d_year#15] - -(28) Filter [codegen id : 5] -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(29) BroadcastExchange -Input [2]: [d_date_sk#14, d_year#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#14] -Join condition: None - -(31) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#11, ss_ext_list_price#12, d_year#15] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_ext_discount_amt#11, ss_ext_list_price#12, d_date_sk#14, d_year#15] - -(32) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#11, ss_ext_list_price#12, d_year#15] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#12 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#24] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#25] - -(33) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#25] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), true, [id=#26] - -(34) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#25] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#12 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#12 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(8,2)))), DecimalType(8,2), true)))#27] -Results [3]: [c_customer_id#2 AS customer_id#28, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#29, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#12 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(8,2)))), DecimalType(8,2), true)))#27,18,2) AS year_total#30] - -(35) BroadcastExchange -Input [3]: [customer_id#28, customer_preferred_cust_flag#29, year_total#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#31] - -(36) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#28] -Join condition: None - -(37) Project [codegen id : 16] -Output [4]: [customer_id#21, year_total#22, customer_preferred_cust_flag#29, year_total#30] -Input [5]: [customer_id#21, year_total#22, customer_id#28, customer_preferred_cust_flag#29, year_total#30] - -(38) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(39) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] - -(40) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(41) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#32, ws_bill_customer_sk#33, ws_ext_discount_amt#34, ws_ext_list_price#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(42) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_date_sk#32, ws_bill_customer_sk#33, ws_ext_discount_amt#34, ws_ext_list_price#35] - -(43) Filter [codegen id : 8] -Input [4]: [ws_sold_date_sk#32, ws_bill_customer_sk#33, ws_ext_discount_amt#34, ws_ext_list_price#35] -Condition : (isnotnull(ws_bill_customer_sk#33) AND isnotnull(ws_sold_date_sk#32)) - -(44) BroadcastExchange -Input [4]: [ws_sold_date_sk#32, ws_bill_customer_sk#33, ws_ext_discount_amt#34, ws_ext_list_price#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#36] - -(45) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#33] -Join condition: None - -(46) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#32, ws_ext_discount_amt#34, ws_ext_list_price#35] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#32, ws_bill_customer_sk#33, ws_ext_discount_amt#34, ws_ext_list_price#35] - -(47) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#14, d_year#15] - -(48) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#14] -Join condition: None - -(49) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#34, ws_ext_list_price#35, d_year#15] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#32, ws_ext_discount_amt#34, ws_ext_list_price#35, d_date_sk#14, d_year#15] - -(50) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#34, ws_ext_list_price#35, d_year#15] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#35 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#34 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#37] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15, sum#38] - -(51) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15, sum#38] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15, 5), true, [id=#39] - -(52) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15, sum#38] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#35 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#34 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#35 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#34 as decimal(8,2)))), DecimalType(8,2), true)))#40] -Results [2]: [c_customer_id#2 AS customer_id#41, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#35 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#34 as decimal(8,2)))), DecimalType(8,2), true)))#40,18,2) AS year_total#42] - -(53) Filter [codegen id : 11] -Input [2]: [customer_id#41, year_total#42] -Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) - -(54) Project [codegen id : 11] -Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] -Input [2]: [customer_id#41, year_total#42] - -(55) BroadcastExchange -Input [2]: [customer_id#43, year_total#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] - -(56) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#43] -Join condition: None - -(57) Project [codegen id : 16] -Output [5]: [customer_id#21, year_total#22, customer_preferred_cust_flag#29, year_total#30, year_total#44] -Input [6]: [customer_id#21, year_total#22, customer_preferred_cust_flag#29, year_total#30, customer_id#43, year_total#44] - -(58) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(59) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] - -(60) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(61) ReusedExchange [Reuses operator id: 44] -Output [4]: [ws_sold_date_sk#32, ws_bill_customer_sk#33, ws_ext_discount_amt#34, ws_ext_list_price#35] - -(62) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#33] -Join condition: None - -(63) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#32, ws_ext_discount_amt#34, ws_ext_list_price#35] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#32, ws_bill_customer_sk#33, ws_ext_discount_amt#34, ws_ext_list_price#35] - -(64) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#14, d_year#15] - -(65) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#14] -Join condition: None - -(66) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#34, ws_ext_list_price#35, d_year#15] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#32, ws_ext_discount_amt#34, ws_ext_list_price#35, d_date_sk#14, d_year#15] - -(67) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#34, ws_ext_list_price#35, d_year#15] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#35 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#34 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#46] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15, sum#47] - -(68) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15, sum#47] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15, 5), true, [id=#48] - -(69) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15, sum#47] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#15] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#35 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#34 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#35 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#34 as decimal(8,2)))), DecimalType(8,2), true)))#49] -Results [2]: [c_customer_id#2 AS customer_id#50, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#35 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#34 as decimal(8,2)))), DecimalType(8,2), true)))#49,18,2) AS year_total#51] - -(70) BroadcastExchange -Input [2]: [customer_id#50, year_total#51] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#52] - -(71) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#50] -Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#51) / promote_precision(year_total#44)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#30) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE null END) - -(72) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#29] -Input [7]: [customer_id#21, year_total#22, customer_preferred_cust_flag#29, year_total#30, year_total#44, customer_id#50, year_total#51] - -(73) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#29] -Arguments: 100, [customer_preferred_cust_flag#29 ASC NULLS FIRST], [customer_preferred_cust_flag#29] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q11/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q11/simplified.txt deleted file mode 100644 index 3850cea62..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q11/simplified.txt +++ /dev/null @@ -1,108 +0,0 @@ -TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (16) - Project [customer_preferred_cust_flag] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_sold_date_sk,ss_ext_discount_amt,ss_ext_list_price] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,customer_preferred_cust_flag,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_sold_date_sk,ss_ext_discount_amt,ss_ext_list_price] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - ReusedExchange [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - Project [customer_id,year_total] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #8 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_sold_date_sk,ws_ext_discount_amt,ws_ext_list_price] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_sold_date_sk,ws_ext_discount_amt,ws_ext_list_price] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - ReusedExchange [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] #9 - InputAdapter - ReusedExchange [d_date_sk,d_year] #6 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q12/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q12/explain.txt deleted file mode 100644 index f786839e8..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q12/explain.txt +++ /dev/null @@ -1,137 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * Sort (21) - +- Exchange (20) - +- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * BroadcastHashJoin Inner BuildRight (15) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.web_sales (1) - : +- BroadcastExchange (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.item (4) - +- BroadcastExchange (14) - +- * Project (13) - +- * Filter (12) - +- * ColumnarToRow (11) - +- Scan parquet default.date_dim (10) - - -(1) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] -Condition : (isnotnull(ws_item_sk#2) AND isnotnull(ws_sold_date_sk#1)) - -(4) Scan parquet default.item -Output [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] - -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Condition : (i_category#9 IN (Sports,Books,Home) AND isnotnull(i_item_sk#4)) - -(7) BroadcastExchange -Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#4] -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Input [9]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] - -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] - -(12) Filter [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10644)) AND (d_date#12 <= 10674)) AND isnotnull(d_date_sk#11)) - -(13) Project [codegen id : 2] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_date#12] - -(14) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#11] -Join condition: None - -(16) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Input [8]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9, d_date_sk#11] - -(17) HashAggregate [codegen id : 3] -Input [6]: [ws_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Keys [5]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#14] -Results [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#15] - -(18) Exchange -Input [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#15] -Arguments: hashpartitioning(i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, 5), true, [id=#16] - -(19) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#15] -Keys [5]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#3))#17] -Results [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#17,17,2) AS _w0#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#17,17,2) AS _w1#20, i_item_id#5] - -(20) Exchange -Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5] -Arguments: hashpartitioning(i_class#8, 5), true, [id=#21] - -(21) Sort [codegen id : 5] -Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5] -Arguments: [i_class#8 ASC NULLS FIRST], false, 0 - -(22) Window -Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5] -Arguments: [sum(_w1#20) windowspecdefinition(i_class#8, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#22], [i_class#8] - -(23) Project [codegen id : 6] -Output [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#19) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#22)), DecimalType(38,17), true) AS revenueratio#23, i_item_id#5] -Input [9]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5, _we0#22] - -(24) TakeOrderedAndProject -Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, revenueratio#23, i_item_id#5] -Arguments: 100, [i_category#9 ASC NULLS FIRST, i_class#8 ASC NULLS FIRST, i_item_id#5 ASC NULLS FIRST, i_item_desc#6 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, revenueratio#23] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q12/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q12/simplified.txt deleted file mode 100644 index f67bfe65f..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q12/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w1,i_class] - WholeStageCodegen (5) - Sort [i_class] - InputAdapter - Exchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,_w1,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q13/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q13/explain.txt deleted file mode 100644 index d0c4e0e49..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q13/explain.txt +++ /dev/null @@ -1,216 +0,0 @@ -== Physical Plan == -* HashAggregate (38) -+- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.store (4) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.customer_address (10) - : : +- BroadcastExchange (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.date_dim (17) - : +- BroadcastExchange (27) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_demographics (24) - +- BroadcastExchange (33) - +- * Filter (32) - +- * ColumnarToRow (31) - +- Scan parquet default.household_demographics (30) - - -(1) Scan parquet default.store_sales -Output [10]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [10]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] - -(3) Filter [codegen id : 6] -Input [10]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] -Condition : ((((((isnotnull(ss_store_sk#5) AND isnotnull(ss_addr_sk#4)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_cdemo_sk#2)) AND isnotnull(ss_hdemo_sk#3)) AND ((((ss_net_profit#10 >= 100.00) AND (ss_net_profit#10 <= 200.00)) OR ((ss_net_profit#10 >= 150.00) AND (ss_net_profit#10 <= 300.00))) OR ((ss_net_profit#10 >= 50.00) AND (ss_net_profit#10 <= 250.00)))) AND ((((ss_sales_price#7 >= 100.00) AND (ss_sales_price#7 <= 150.00)) OR ((ss_sales_price#7 >= 50.00) AND (ss_sales_price#7 <= 100.00))) OR ((ss_sales_price#7 >= 150.00) AND (ss_sales_price#7 <= 200.00)))) - -(4) Scan parquet default.store -Output [1]: [s_store_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#11] - -(6) Filter [codegen id : 1] -Input [1]: [s_store_sk#11] -Condition : isnotnull(s_store_sk#11) - -(7) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#11] -Join condition: None - -(9) Project [codegen id : 6] -Output [9]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] -Input [11]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, s_store_sk#11] - -(10) Scan parquet default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [TX,OH]),In(ca_state, [OR,NM,KY])),In(ca_state, [VA,TX,MS]))] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] - -(12) Filter [codegen id : 2] -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((ca_state#14 IN (TX,OH) OR ca_state#14 IN (OR,NM,KY)) OR ca_state#14 IN (VA,TX,MS))) - -(13) Project [codegen id : 2] -Output [2]: [ca_address_sk#13, ca_state#14] -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] - -(14) BroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#4] -Right keys [1]: [ca_address_sk#13] -Join condition: ((((ca_state#14 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#14 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#14 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) - -(16) Project [codegen id : 6] -Output [7]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [11]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#13, ca_state#14] - -(17) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#17, d_year#18] - -(19) Filter [codegen id : 3] -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(20) Project [codegen id : 3] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_year#18] - -(21) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] - -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#17] -Join condition: None - -(23) Project [codegen id : 6] -Output [6]: [ss_cdemo_sk#2, ss_hdemo_sk#3, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [8]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, d_date_sk#17] - -(24) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -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)))] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] - -(26) Filter [codegen id : 4] -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Condition : (isnotnull(cd_demo_sk#20) AND ((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree)) OR ((cd_marital_status#21 = S) AND (cd_education_status#22 = College))) OR ((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree)))) - -(27) BroadcastExchange -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join condition: ((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree)) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) OR ((((cd_marital_status#21 = S) AND (cd_education_status#22 = College)) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00))) OR ((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree)) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00))) - -(29) Project [codegen id : 6] -Output [7]: [ss_hdemo_sk#3, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, cd_marital_status#21, cd_education_status#22] -Input [9]: [ss_cdemo_sk#2, ss_hdemo_sk#3, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] - -(30) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#24, hd_dep_count#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1)),EqualTo(hd_dep_count,1))] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 5] -Input [2]: [hd_demo_sk#24, hd_dep_count#25] - -(32) Filter [codegen id : 5] -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Condition : (isnotnull(hd_demo_sk#24) AND (((hd_dep_count#25 = 3) OR (hd_dep_count#25 = 1)) OR (hd_dep_count#25 = 1))) - -(33) BroadcastExchange -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] - -(34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#24] -Join condition: (((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree)) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) AND (hd_dep_count#25 = 3)) OR (((((cd_marital_status#21 = S) AND (cd_education_status#22 = College)) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00)) AND (hd_dep_count#25 = 1))) OR (((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree)) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00)) AND (hd_dep_count#25 = 1))) - -(35) Project [codegen id : 6] -Output [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [9]: [ss_hdemo_sk#3, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, cd_marital_status#21, cd_education_status#22, hd_demo_sk#24, hd_dep_count#25] - -(36) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Keys: [] -Functions [4]: [partial_avg(cast(ss_quantity#6 as bigint)), partial_avg(UnscaledValue(ss_ext_sales_price#8)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#9)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#9))] -Aggregate Attributes [7]: [sum#27, count#28, sum#29, count#30, sum#31, count#32, sum#33] -Results [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] - -(37) Exchange -Input [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] -Arguments: SinglePartition, true, [id=#41] - -(38) HashAggregate [codegen id : 7] -Input [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] -Keys: [] -Functions [4]: [avg(cast(ss_quantity#6 as bigint)), avg(UnscaledValue(ss_ext_sales_price#8)), avg(UnscaledValue(ss_ext_wholesale_cost#9)), sum(UnscaledValue(ss_ext_wholesale_cost#9))] -Aggregate Attributes [4]: [avg(cast(ss_quantity#6 as bigint))#42, avg(UnscaledValue(ss_ext_sales_price#8))#43, avg(UnscaledValue(ss_ext_wholesale_cost#9))#44, sum(UnscaledValue(ss_ext_wholesale_cost#9))#45] -Results [4]: [avg(cast(ss_quantity#6 as bigint))#42 AS avg(ss_quantity)#46, cast((avg(UnscaledValue(ss_ext_sales_price#8))#43 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#47, cast((avg(UnscaledValue(ss_ext_wholesale_cost#9))#44 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#48, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#9))#45,17,2) AS sum(ss_ext_wholesale_cost)#49] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q13/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q13/simplified.txt deleted file mode 100644 index 6e3a322d8..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q13/simplified.txt +++ /dev/null @@ -1,57 +0,0 @@ -WholeStageCodegen (7) - HashAggregate [sum,count,sum,count,sum,count,sum] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] - InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] - Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_addr_sk,ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [ca_address_sk,ca_state] - Filter [ca_country,ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [cd_demo_sk,cd_marital_status,cd_education_status] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Filter [hd_demo_sk,hd_dep_count] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14a/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14a/explain.txt deleted file mode 100644 index 3f0cc9e7a..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14a/explain.txt +++ /dev/null @@ -1,798 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (115) -+- * HashAggregate (114) - +- Exchange (113) - +- * HashAggregate (112) - +- * Expand (111) - +- Union (110) - :- * Project (77) - : +- * Filter (76) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * HashAggregate (52) - : : : +- * HashAggregate (51) - : : : +- * HashAggregate (50) - : : : +- Exchange (49) - : : : +- * HashAggregate (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (63) - : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : :- * Filter (60) - : : : +- * ColumnarToRow (59) - : : : +- Scan parquet default.item (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (70) - : +- * Project (69) - : +- * Filter (68) - : +- * ColumnarToRow (67) - : +- Scan parquet default.date_dim (66) - :- * Project (93) - : +- * Filter (92) - : +- * HashAggregate (91) - : +- Exchange (90) - : +- * HashAggregate (89) - : +- * Project (88) - : +- * BroadcastHashJoin Inner BuildRight (87) - : :- * Project (85) - : : +- * BroadcastHashJoin Inner BuildRight (84) - : : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : : :- * Filter (80) - : : : : +- * ColumnarToRow (79) - : : : : +- Scan parquet default.catalog_sales (78) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - +- * Project (109) - +- * Filter (108) - +- * HashAggregate (107) - +- Exchange (106) - +- * HashAggregate (105) - +- * Project (104) - +- * BroadcastHashJoin Inner BuildRight (103) - :- * Project (101) - : +- * BroadcastHashJoin Inner BuildRight (100) - : :- * BroadcastHashJoin LeftSemi BuildRight (98) - : : :- * Filter (96) - : : : +- * ColumnarToRow (95) - : : : +- Scan parquet default.web_sales (94) - : : +- ReusedExchange (97) - : +- ReusedExchange (99) - +- ReusedExchange (102) - - -(1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) - -(4) Scan parquet default.item -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(6) Filter [codegen id : 11] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : ((isnotnull(i_brand_id#6) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) - -(7) Scan parquet default.store_sales -Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 9] -Input [2]: [ss_sold_date_sk#1, ss_item_sk#2] - -(9) Filter [codegen id : 9] -Input [2]: [ss_sold_date_sk#1, ss_item_sk#2] -Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) - -(10) Scan parquet default.item -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) - -(13) BroadcastExchange -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#9] - -(14) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(15) Project [codegen id : 9] -Output [4]: [ss_sold_date_sk#1, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#11] - -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#11] -Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) - -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#11] - -(20) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(21) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(22) Project [codegen id : 9] -Output [3]: [i_brand_id#6 AS brand_id#13, i_class_id#7 AS class_id#14, i_category_id#8 AS category_id#15] -Input [5]: [ss_sold_date_sk#1, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] - -(23) Scan parquet default.catalog_sales -Output [2]: [cs_sold_date_sk#16, cs_item_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_sold_date_sk#16, cs_item_sk#17] - -(25) Filter [codegen id : 5] -Input [2]: [cs_sold_date_sk#16, cs_item_sk#17] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_sold_date_sk#16)) - -(26) Scan parquet default.item -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : isnotnull(i_item_sk#5) - -(29) BroadcastExchange -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] - -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#16, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [6]: [cs_sold_date_sk#16, cs_item_sk#17, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#10] - -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Input [5]: [cs_sold_date_sk#16, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] - -(35) BroadcastExchange -Input [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#19] - -(36) BroadcastHashJoin [codegen id : 9] -Left keys [6]: [coalesce(brand_id#13, 0), isnull(brand_id#13), coalesce(class_id#14, 0), isnull(class_id#14), coalesce(category_id#15, 0), isnull(category_id#15)] -Right keys [6]: [coalesce(i_brand_id#6, 0), isnull(i_brand_id#6), coalesce(i_class_id#7, 0), isnull(i_class_id#7), coalesce(i_category_id#8, 0), isnull(i_category_id#8)] -Join condition: None - -(37) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#20, ws_item_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 8] -Input [2]: [ws_sold_date_sk#20, ws_item_sk#21] - -(39) Filter [codegen id : 8] -Input [2]: [ws_sold_date_sk#20, ws_item_sk#21] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_sold_date_sk#20)) - -(40) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_item_sk#21] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(42) Project [codegen id : 8] -Output [4]: [ws_sold_date_sk#20, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [6]: [ws_sold_date_sk#20, ws_item_sk#21, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(43) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#10] - -(44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_date_sk#20] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(45) Project [codegen id : 8] -Output [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Input [5]: [ws_sold_date_sk#20, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] - -(46) BroadcastExchange -Input [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#22] - -(47) BroadcastHashJoin [codegen id : 9] -Left keys [6]: [coalesce(brand_id#13, 0), isnull(brand_id#13), coalesce(class_id#14, 0), isnull(class_id#14), coalesce(category_id#15, 0), isnull(category_id#15)] -Right keys [6]: [coalesce(i_brand_id#6, 0), isnull(i_brand_id#6), coalesce(i_class_id#7, 0), isnull(i_class_id#7), coalesce(i_category_id#8, 0), isnull(i_category_id#8)] -Join condition: None - -(48) HashAggregate [codegen id : 9] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(49) Exchange -Input [3]: [brand_id#13, class_id#14, category_id#15] -Arguments: hashpartitioning(brand_id#13, class_id#14, category_id#15, 5), true, [id=#23] - -(50) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange -Input [3]: [brand_id#13, class_id#14, category_id#15] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] - -(54) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Right keys [3]: [brand_id#13, class_id#14, category_id#15] -Join condition: None - -(55) Project [codegen id : 11] -Output [1]: [i_item_sk#5 AS ss_item_sk#25] -Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] - -(56) BroadcastExchange -Input [1]: [ss_item_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] - -(57) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [ss_item_sk#25] -Join condition: None - -(58) Scan parquet default.item -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(60) Filter [codegen id : 23] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : isnotnull(i_item_sk#5) - -(61) ReusedExchange [Reuses operator id: 56] -Output [1]: [ss_item_sk#25] - -(62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#5] -Right keys [1]: [ss_item_sk#25] -Join condition: None - -(63) BroadcastExchange -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] - -(64) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(65) Project [codegen id : 25] -Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(66) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_moy#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#10, d_year#11, d_moy#28] - -(68) Filter [codegen id : 24] -Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#28)) AND (d_year#11 = 2001)) AND (d_moy#28 = 11)) AND isnotnull(d_date_sk#10)) - -(69) Project [codegen id : 24] -Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#11, d_moy#28] - -(70) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] - -(71) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(72) Project [codegen id : 25] -Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] - -(73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] -Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#30, isEmpty#31, count#32] -Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] - -(74) Exchange -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] -Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#36] - -(75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] -Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37, count(1)#38] -Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#39, count(1)#38 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] - -(76) Filter [codegen id : 26] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41 as decimal(32,6)) > cast(Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) - -(77) Project [codegen id : 26] -Output [6]: [sales#39, number_sales#40, store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] - -(78) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] - -(80) Filter [codegen id : 51] -Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_sold_date_sk#16)) - -(81) ReusedExchange [Reuses operator id: 56] -Output [1]: [ss_item_sk#25] - -(82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [ss_item_sk#25] -Join condition: None - -(83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(85) Project [codegen id : 51] -Output [6]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [8]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#10] - -(87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(88) Project [codegen id : 51] -Output [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [7]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] - -(89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] -Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] -Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] - -(90) Exchange -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] -Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#53] - -(91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] -Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] -Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] - -(92) Filter [codegen id : 52] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) - -(93) Project [codegen id : 52] -Output [6]: [sales#56, number_sales#57, catalog AS channel#59, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] - -(94) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] - -(96) Filter [codegen id : 77] -Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_sold_date_sk#20)) - -(97) ReusedExchange [Reuses operator id: 56] -Output [1]: [ss_item_sk#25] - -(98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#21] -Right keys [1]: [ss_item_sk#25] -Join condition: None - -(99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#21] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(101) Project [codegen id : 77] -Output [6]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [8]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#10] - -(103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#20] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(104) Project [codegen id : 77] -Output [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [7]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] - -(105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] -Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#62, isEmpty#63, count#64] -Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] - -(106) Exchange -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] -Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#68] - -(107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] -Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69, count(1)#70] -Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#71, count(1)#70 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] - -(108) Filter [codegen id : 78] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) - -(109) Project [codegen id : 78] -Output [6]: [sales#71, number_sales#72, web AS channel#74, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] - -(110) Union - -(111) Expand [codegen id : 79] -Input [6]: [sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: [List(sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 0), List(sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, null, 1), List(sales#39, number_sales#40, channel#44, i_brand_id#6, null, null, 3), List(sales#39, number_sales#40, channel#44, null, null, null, 7), List(sales#39, number_sales#40, null, null, null, null, 15)], [sales#39, number_sales#40, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] - -(112) HashAggregate [codegen id : 79] -Input [7]: [sales#39, number_sales#40, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] -Keys [5]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] -Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] -Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82] -Results [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] - -(113) Exchange -Input [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] -Arguments: hashpartitioning(channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, 5), true, [id=#86] - -(114) HashAggregate [codegen id : 80] -Input [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] -Keys [5]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] -Functions [2]: [sum(sales#39), sum(number_sales#40)] -Aggregate Attributes [2]: [sum(sales#39)#87, sum(number_sales#40)#88] -Results [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales#39)#87 AS sum(sales)#89, sum(number_sales#40)#88 AS sum(number_sales)#90] - -(115) TakeOrderedAndProject -Input [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales)#89, sum(number_sales)#90] -Arguments: 100, [channel#75 ASC NULLS FIRST, i_brand_id#76 ASC NULLS FIRST, i_class_id#77 ASC NULLS FIRST, i_category_id#78 ASC NULLS FIRST], [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales)#89, sum(number_sales)#90] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* HashAggregate (141) -+- Exchange (140) - +- * HashAggregate (139) - +- Union (138) - :- * Project (125) - : +- * BroadcastHashJoin Inner BuildRight (124) - : :- * Filter (118) - : : +- * ColumnarToRow (117) - : : +- Scan parquet default.store_sales (116) - : +- BroadcastExchange (123) - : +- * Project (122) - : +- * Filter (121) - : +- * ColumnarToRow (120) - : +- Scan parquet default.date_dim (119) - :- * Project (131) - : +- * BroadcastHashJoin Inner BuildRight (130) - : :- * Filter (128) - : : +- * ColumnarToRow (127) - : : +- Scan parquet default.catalog_sales (126) - : +- ReusedExchange (129) - +- * Project (137) - +- * BroadcastHashJoin Inner BuildRight (136) - :- * Filter (134) - : +- * ColumnarToRow (133) - : +- Scan parquet default.web_sales (132) - +- ReusedExchange (135) - - -(116) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(117) ColumnarToRow [codegen id : 2] -Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] - -(118) Filter [codegen id : 2] -Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -Condition : isnotnull(ss_sold_date_sk#1) - -(119) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_year#11] - -(121) Filter [codegen id : 1] -Input [2]: [d_date_sk#10, d_year#11] -Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) - -(122) Project [codegen id : 1] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#11] - -(123) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#91] - -(124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(125) Project [codegen id : 2] -Output [2]: [ss_quantity#3 AS quantity#92, ss_list_price#4 AS list_price#93] -Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] - -(126) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] - -(128) Filter [codegen id : 4] -Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] -Condition : isnotnull(cs_sold_date_sk#16) - -(129) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#10] - -(130) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(131) Project [codegen id : 4] -Output [2]: [cs_quantity#45 AS quantity#94, cs_list_price#46 AS list_price#95] -Input [4]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, d_date_sk#10] - -(132) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(133) ColumnarToRow [codegen id : 6] -Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] - -(134) Filter [codegen id : 6] -Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] -Condition : isnotnull(ws_sold_date_sk#20) - -(135) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#10] - -(136) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#20] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(137) Project [codegen id : 6] -Output [2]: [ws_quantity#60 AS quantity#96, ws_list_price#61 AS list_price#97] -Input [4]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, d_date_sk#10] - -(138) Union - -(139) HashAggregate [codegen id : 7] -Input [2]: [quantity#92, list_price#93] -Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#98, count#99] -Results [2]: [sum#100, count#101] - -(140) Exchange -Input [2]: [sum#100, count#101] -Arguments: SinglePartition, true, [id=#102] - -(141) HashAggregate [codegen id : 8] -Input [2]: [sum#100, count#101] -Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#103] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#103 AS average_sales#104] - -Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] - -Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14a/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14a/simplified.txt deleted file mode 100644 index dfa8c1bcc..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14a/simplified.txt +++ /dev/null @@ -1,214 +0,0 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] - InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - InputAdapter - Union - WholeStageCodegen (26) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] - Subquery #1 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] - InputAdapter - Exchange #13 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] - InputAdapter - ReusedExchange [d_date_sk] #14 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] - InputAdapter - ReusedExchange [d_date_sk] #14 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (24) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (52) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #12 - WholeStageCodegen (78) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_quantity,ws_list_price] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #12 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14b/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14b/explain.txt deleted file mode 100644 index 1f31ded51..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14b/explain.txt +++ /dev/null @@ -1,763 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (100) -+- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (77) - : +- * Filter (76) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * HashAggregate (52) - : : : +- * HashAggregate (51) - : : : +- * HashAggregate (50) - : : : +- Exchange (49) - : : : +- * HashAggregate (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (63) - : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : :- * Filter (60) - : : : +- * ColumnarToRow (59) - : : : +- Scan parquet default.item (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (70) - : +- * Project (69) - : +- * Filter (68) - : +- * ColumnarToRow (67) - : +- Scan parquet default.date_dim (66) - +- BroadcastExchange (98) - +- * Project (97) - +- * Filter (96) - +- * HashAggregate (95) - +- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * BroadcastHashJoin Inner BuildRight (91) - :- * Project (85) - : +- * BroadcastHashJoin Inner BuildRight (84) - : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : :- * Filter (80) - : : : +- * ColumnarToRow (79) - : : : +- Scan parquet default.store_sales (78) - : : +- ReusedExchange (81) - : +- ReusedExchange (83) - +- BroadcastExchange (90) - +- * Project (89) - +- * Filter (88) - +- * ColumnarToRow (87) - +- Scan parquet default.date_dim (86) - - -(1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) - -(4) Scan parquet default.item -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(6) Filter [codegen id : 11] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : ((isnotnull(i_brand_id#6) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) - -(7) Scan parquet default.store_sales -Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 9] -Input [2]: [ss_sold_date_sk#1, ss_item_sk#2] - -(9) Filter [codegen id : 9] -Input [2]: [ss_sold_date_sk#1, ss_item_sk#2] -Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) - -(10) Scan parquet default.item -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) - -(13) BroadcastExchange -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#9] - -(14) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(15) Project [codegen id : 9] -Output [4]: [ss_sold_date_sk#1, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#11] - -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#11] -Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) - -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#11] - -(20) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(21) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(22) Project [codegen id : 9] -Output [3]: [i_brand_id#6 AS brand_id#13, i_class_id#7 AS class_id#14, i_category_id#8 AS category_id#15] -Input [5]: [ss_sold_date_sk#1, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] - -(23) Scan parquet default.catalog_sales -Output [2]: [cs_sold_date_sk#16, cs_item_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_sold_date_sk#16, cs_item_sk#17] - -(25) Filter [codegen id : 5] -Input [2]: [cs_sold_date_sk#16, cs_item_sk#17] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_sold_date_sk#16)) - -(26) Scan parquet default.item -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : isnotnull(i_item_sk#5) - -(29) BroadcastExchange -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] - -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#16, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [6]: [cs_sold_date_sk#16, cs_item_sk#17, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#10] - -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Input [5]: [cs_sold_date_sk#16, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] - -(35) BroadcastExchange -Input [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#19] - -(36) BroadcastHashJoin [codegen id : 9] -Left keys [6]: [coalesce(brand_id#13, 0), isnull(brand_id#13), coalesce(class_id#14, 0), isnull(class_id#14), coalesce(category_id#15, 0), isnull(category_id#15)] -Right keys [6]: [coalesce(i_brand_id#6, 0), isnull(i_brand_id#6), coalesce(i_class_id#7, 0), isnull(i_class_id#7), coalesce(i_category_id#8, 0), isnull(i_category_id#8)] -Join condition: None - -(37) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#20, ws_item_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 8] -Input [2]: [ws_sold_date_sk#20, ws_item_sk#21] - -(39) Filter [codegen id : 8] -Input [2]: [ws_sold_date_sk#20, ws_item_sk#21] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_sold_date_sk#20)) - -(40) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_item_sk#21] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(42) Project [codegen id : 8] -Output [4]: [ws_sold_date_sk#20, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [6]: [ws_sold_date_sk#20, ws_item_sk#21, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(43) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#10] - -(44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_date_sk#20] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(45) Project [codegen id : 8] -Output [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Input [5]: [ws_sold_date_sk#20, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] - -(46) BroadcastExchange -Input [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#22] - -(47) BroadcastHashJoin [codegen id : 9] -Left keys [6]: [coalesce(brand_id#13, 0), isnull(brand_id#13), coalesce(class_id#14, 0), isnull(class_id#14), coalesce(category_id#15, 0), isnull(category_id#15)] -Right keys [6]: [coalesce(i_brand_id#6, 0), isnull(i_brand_id#6), coalesce(i_class_id#7, 0), isnull(i_class_id#7), coalesce(i_category_id#8, 0), isnull(i_category_id#8)] -Join condition: None - -(48) HashAggregate [codegen id : 9] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(49) Exchange -Input [3]: [brand_id#13, class_id#14, category_id#15] -Arguments: hashpartitioning(brand_id#13, class_id#14, category_id#15, 5), true, [id=#23] - -(50) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange -Input [3]: [brand_id#13, class_id#14, category_id#15] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] - -(54) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Right keys [3]: [brand_id#13, class_id#14, category_id#15] -Join condition: None - -(55) Project [codegen id : 11] -Output [1]: [i_item_sk#5 AS ss_item_sk#25] -Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] - -(56) BroadcastExchange -Input [1]: [ss_item_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] - -(57) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [ss_item_sk#25] -Join condition: None - -(58) Scan parquet default.item -Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(60) Filter [codegen id : 23] -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) - -(61) ReusedExchange [Reuses operator id: 56] -Output [1]: [ss_item_sk#25] - -(62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#5] -Right keys [1]: [ss_item_sk#25] -Join condition: None - -(63) BroadcastExchange -Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] - -(64) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(65) Project [codegen id : 25] -Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] - -(66) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_week_seq#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#10, d_week_seq#28] - -(68) Filter [codegen id : 24] -Input [2]: [d_date_sk#10, d_week_seq#28] -Condition : ((isnotnull(d_week_seq#28) AND (d_week_seq#28 = Subquery scalar-subquery#29, [id=#30])) AND isnotnull(d_date_sk#10)) - -(69) Project [codegen id : 24] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_week_seq#28] - -(70) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] - -(71) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(72) Project [codegen id : 25] -Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] - -(73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] -Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] -Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] - -(74) Exchange -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] -Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#38] - -(75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] -Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] -Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] - -(76) Filter [codegen id : 52] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) - -(77) Project [codegen id : 52] -Output [6]: [store AS channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] - -(78) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] - -(80) Filter [codegen id : 50] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) - -(81) ReusedExchange [Reuses operator id: 56] -Output [1]: [ss_item_sk#25] - -(82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [ss_item_sk#25] -Join condition: None - -(83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] - -(84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#47] -Join condition: None - -(85) Project [codegen id : 50] -Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] -Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] - -(86) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_week_seq#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#10, d_week_seq#28] - -(88) Filter [codegen id : 49] -Input [2]: [d_date_sk#10, d_week_seq#28] -Condition : ((isnotnull(d_week_seq#28) AND (d_week_seq#28 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#10)) - -(89) Project [codegen id : 49] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_week_seq#28] - -(90) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] - -(91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(92) Project [codegen id : 50] -Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] -Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50, d_date_sk#10] - -(93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] -Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] -Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] - -(94) Exchange -Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] -Arguments: hashpartitioning(i_brand_id#48, i_class_id#49, i_category_id#50, 5), true, [id=#60] - -(95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] -Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] -Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] - -(96) Filter [codegen id : 51] -Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) - -(97) Project [codegen id : 51] -Output [6]: [store AS channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] -Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] - -(98) BroadcastExchange -Input [6]: [channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#67] - -(99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] -Right keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] -Join condition: None - -(100) TakeOrderedAndProject -Input [12]: [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] -Arguments: 100, [i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* HashAggregate (126) -+- Exchange (125) - +- * HashAggregate (124) - +- Union (123) - :- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * Filter (103) - : : +- * ColumnarToRow (102) - : : +- Scan parquet default.store_sales (101) - : +- BroadcastExchange (108) - : +- * Project (107) - : +- * Filter (106) - : +- * ColumnarToRow (105) - : +- Scan parquet default.date_dim (104) - :- * Project (116) - : +- * BroadcastHashJoin Inner BuildRight (115) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.catalog_sales (111) - : +- ReusedExchange (114) - +- * Project (122) - +- * BroadcastHashJoin Inner BuildRight (121) - :- * Filter (119) - : +- * ColumnarToRow (118) - : +- Scan parquet default.web_sales (117) - +- ReusedExchange (120) - - -(101) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] - -(103) Filter [codegen id : 2] -Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -Condition : isnotnull(ss_sold_date_sk#1) - -(104) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_year#11] - -(106) Filter [codegen id : 1] -Input [2]: [d_date_sk#10, d_year#11] -Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) - -(107) Project [codegen id : 1] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#11] - -(108) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] - -(109) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(110) Project [codegen id : 2] -Output [2]: [ss_quantity#3 AS quantity#69, ss_list_price#4 AS list_price#70] -Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] - -(111) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(112) ColumnarToRow [codegen id : 4] -Input [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] - -(113) Filter [codegen id : 4] -Input [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] -Condition : isnotnull(cs_sold_date_sk#16) - -(114) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#10] - -(115) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(116) Project [codegen id : 4] -Output [2]: [cs_quantity#71 AS quantity#73, cs_list_price#72 AS list_price#74] -Input [4]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72, d_date_sk#10] - -(117) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(118) ColumnarToRow [codegen id : 6] -Input [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] - -(119) Filter [codegen id : 6] -Input [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] -Condition : isnotnull(ws_sold_date_sk#20) - -(120) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#10] - -(121) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#20] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(122) Project [codegen id : 6] -Output [2]: [ws_quantity#75 AS quantity#77, ws_list_price#76 AS list_price#78] -Input [4]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76, d_date_sk#10] - -(123) Union - -(124) HashAggregate [codegen id : 7] -Input [2]: [quantity#69, list_price#70] -Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#79, count#80] -Results [2]: [sum#81, count#82] - -(125) Exchange -Input [2]: [sum#81, count#82] -Arguments: SinglePartition, true, [id=#83] - -(126) HashAggregate [codegen id : 8] -Input [2]: [sum#81, count#82] -Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84 AS average_sales#85] - -Subquery:2 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#29, [id=#30] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) - - -(127) Scan parquet default.date_dim -Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] - -(129) Filter [codegen id : 1] -Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 2000)) AND (d_moy#86 = 12)) AND (d_dom#87 = 11)) - -(130) Project [codegen id : 1] -Output [1]: [d_week_seq#28] -Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] - -Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] - -Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* Project (134) -+- * Filter (133) - +- * ColumnarToRow (132) - +- Scan parquet default.date_dim (131) - - -(131) Scan parquet default.date_dim -Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(132) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] - -(133) Filter [codegen id : 1] -Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1999)) AND (d_moy#86 = 12)) AND (d_dom#87 = 11)) - -(134) Project [codegen id : 1] -Output [1]: [d_week_seq#28] -Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14b/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14b/simplified.txt deleted file mode 100644 index 7bbf83e3d..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q14b/simplified.txt +++ /dev/null @@ -1,204 +0,0 @@ -TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] - Subquery #2 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] - InputAdapter - Exchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] - InputAdapter - ReusedExchange [d_date_sk] #13 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] - InputAdapter - ReusedExchange [d_date_sk] #13 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #4 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (24) - Project [d_date_sk] - Filter [d_week_seq,d_date_sk] - Subquery #1 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (51) - Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [average_sales] #2 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] - InputAdapter - ReusedExchange [ss_item_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (49) - Project [d_date_sk] - Filter [d_week_seq,d_date_sk] - Subquery #3 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q15/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q15/explain.txt deleted file mode 100644 index 4dc0abf9a..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q15/explain.txt +++ /dev/null @@ -1,150 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- Exchange (24) - +- * HashAggregate (23) - +- * Project (22) - +- * BroadcastHashJoin Inner BuildRight (21) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.catalog_sales (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.customer (4) - : +- BroadcastExchange (13) - : +- * Filter (12) - : +- * ColumnarToRow (11) - : +- Scan parquet default.customer_address (10) - +- BroadcastExchange (20) - +- * Project (19) - +- * Filter (18) - +- * ColumnarToRow (17) - +- Scan parquet default.date_dim (16) - - -(1) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_sales_price#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_sales_price#3] - -(3) Filter [codegen id : 4] -Input [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_sales_price#3] -Condition : (isnotnull(cs_bill_customer_sk#2) AND isnotnull(cs_sold_date_sk#1)) - -(4) Scan parquet default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] - -(6) Filter [codegen id : 1] -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_customer_sk#4) AND isnotnull(c_current_addr_sk#5)) - -(7) BroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#4] -Join condition: None - -(9) Project [codegen id : 4] -Output [3]: [cs_sold_date_sk#1, cs_sales_price#3, c_current_addr_sk#5] -Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_sales_price#3, c_customer_sk#4, c_current_addr_sk#5] - -(10) Scan parquet default.customer_address -Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] - -(12) Filter [codegen id : 2] -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Condition : isnotnull(ca_address_sk#7) - -(13) BroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#7] -Join condition: ((substr(ca_zip#9, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#8 IN (CA,WA,GA)) OR (cs_sales_price#3 > 500.00)) - -(15) Project [codegen id : 4] -Output [3]: [cs_sold_date_sk#1, cs_sales_price#3, ca_zip#9] -Input [6]: [cs_sold_date_sk#1, cs_sales_price#3, c_current_addr_sk#5, ca_address_sk#7, ca_state#8, ca_zip#9] - -(16) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_qoy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#11, d_year#12, d_qoy#13] - -(18) Filter [codegen id : 3] -Input [3]: [d_date_sk#11, d_year#12, d_qoy#13] -Condition : ((((isnotnull(d_qoy#13) AND isnotnull(d_year#12)) AND (d_qoy#13 = 2)) AND (d_year#12 = 2001)) AND isnotnull(d_date_sk#11)) - -(19) Project [codegen id : 3] -Output [1]: [d_date_sk#11] -Input [3]: [d_date_sk#11, d_year#12, d_qoy#13] - -(20) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#11] -Join condition: None - -(22) Project [codegen id : 4] -Output [2]: [cs_sales_price#3, ca_zip#9] -Input [4]: [cs_sold_date_sk#1, cs_sales_price#3, ca_zip#9, d_date_sk#11] - -(23) HashAggregate [codegen id : 4] -Input [2]: [cs_sales_price#3, ca_zip#9] -Keys [1]: [ca_zip#9] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#3))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [ca_zip#9, sum#16] - -(24) Exchange -Input [2]: [ca_zip#9, sum#16] -Arguments: hashpartitioning(ca_zip#9, 5), true, [id=#17] - -(25) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#9, sum#16] -Keys [1]: [ca_zip#9] -Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#18] -Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#18,17,2) AS sum(cs_sales_price)#19] - -(26) TakeOrderedAndProject -Input [2]: [ca_zip#9, sum(cs_sales_price)#19] -Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#19] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q15/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q15/simplified.txt deleted file mode 100644 index 9cac0b46f..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q15/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (5) - HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - InputAdapter - Exchange [ca_zip] #1 - WholeStageCodegen (4) - HashAggregate [ca_zip,cs_sales_price] [sum,sum] - Project [cs_sales_price,ca_zip] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_sales_price,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] - Project [cs_sold_date_sk,cs_sales_price,c_current_addr_sk] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Filter [cs_bill_customer_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_sales_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q16/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q16/explain.txt deleted file mode 100644 index ea7e29839..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q16/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* Sort (41) -+- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * BroadcastHashJoin LeftAnti BuildRight (13) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Project (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.catalog_sales (4) - : : : +- BroadcastExchange (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.catalog_returns (10) - : : +- BroadcastExchange (18) - : : +- * Project (17) - : : +- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet default.date_dim (14) - : +- BroadcastExchange (25) - : +- * Project (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet default.customer_address (21) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- * ColumnarToRow (29) - +- Scan parquet default.call_center (28) - - -(1) Scan parquet default.catalog_sales -Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(3) Filter [codegen id : 6] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) - -(4) Scan parquet default.catalog_sales -Output [2]: [cs_warehouse_sk#4, cs_order_number#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [cs_warehouse_sk#4, cs_order_number#5] - -(6) Project [codegen id : 1] -Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#8, cs_order_number#5 AS cs_order_number#5#9] -Input [2]: [cs_warehouse_sk#4, cs_order_number#5] - -(7) BroadcastExchange -Input [2]: [cs_warehouse_sk#4#8, cs_order_number#5#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#10] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_order_number#5] -Right keys [1]: [cs_order_number#5#9] -Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#8) - -(9) Project [codegen id : 6] -Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(10) Scan parquet default.catalog_returns -Output [1]: [cr_order_number#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [1]: [cr_order_number#11] - -(12) BroadcastExchange -Input [1]: [cr_order_number#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(13) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_order_number#5] -Right keys [1]: [cr_order_number#11] -Join condition: None - -(14) Scan parquet default.date_dim -Output [2]: [d_date_sk#13, d_date#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#13, d_date#14] - -(16) Filter [codegen id : 3] -Input [2]: [d_date_sk#13, d_date#14] -Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 11719)) AND (d_date#14 <= 11779)) AND isnotnull(d_date_sk#13)) - -(17) Project [codegen id : 3] -Output [1]: [d_date_sk#13] -Input [2]: [d_date_sk#13, d_date#14] - -(18) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] - -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(20) Project [codegen id : 6] -Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#13] - -(21) Scan parquet default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(22) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#16, ca_state#17] - -(23) Filter [codegen id : 4] -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) - -(24) Project [codegen id : 4] -Output [1]: [ca_address_sk#16] -Input [2]: [ca_address_sk#16, ca_state#17] - -(25) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#16] -Join condition: None - -(27) Project [codegen id : 6] -Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] - -(28) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#19, cc_county#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 5] -Input [2]: [cc_call_center_sk#19, cc_county#20] - -(30) Filter [codegen id : 5] -Input [2]: [cc_call_center_sk#19, cc_county#20] -Condition : ((isnotnull(cc_county#20) AND (cc_county#20 = Williamson County)) AND isnotnull(cc_call_center_sk#19)) - -(31) Project [codegen id : 5] -Output [1]: [cc_call_center_sk#19] -Input [2]: [cc_call_center_sk#19, cc_county#20] - -(32) BroadcastExchange -Input [1]: [cc_call_center_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] - -(33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_call_center_sk#3] -Right keys [1]: [cc_call_center_sk#19] -Join condition: None - -(34) Project [codegen id : 6] -Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#19] - -(35) HashAggregate [codegen id : 6] -Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Keys [1]: [cs_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#24, sum#25] - -(36) Exchange -Input [3]: [cs_order_number#5, sum#24, sum#25] -Arguments: hashpartitioning(cs_order_number#5, 5), true, [id=#26] - -(37) HashAggregate [codegen id : 7] -Input [3]: [cs_order_number#5, sum#24, sum#25] -Keys [1]: [cs_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#24, sum#25] - -(38) HashAggregate [codegen id : 7] -Input [3]: [cs_order_number#5, sum#24, sum#25] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#27] -Results [3]: [sum#24, sum#25, count#28] - -(39) Exchange -Input [3]: [sum#24, sum#25, count#28] -Arguments: SinglePartition, true, [id=#29] - -(40) HashAggregate [codegen id : 8] -Input [3]: [sum#24, sum#25, count#28] -Keys: [] -Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#27] -Results [3]: [count(cs_order_number#5)#27 AS order count #30, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#22,17,2) AS total shipping cost #31, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#23,17,2) AS total net profit #32] - -(41) Sort [codegen id : 8] -Input [3]: [order count #30, total shipping cost #31, total net profit #32] -Arguments: [order count #30 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q16/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q16/simplified.txt deleted file mode 100644 index 169f07c2d..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q16/simplified.txt +++ /dev/null @@ -1,62 +0,0 @@ -WholeStageCodegen (8) - Sort [order count ] - HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (7) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - InputAdapter - Exchange [cs_order_number] #2 - WholeStageCodegen (6) - HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] - Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - BroadcastHashJoin [cs_order_number,cr_order_number] - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [cs_warehouse_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Project [cc_call_center_sk] - Filter [cc_county,cc_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.call_center [cc_call_center_sk,cc_county] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q17/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q17/explain.txt deleted file mode 100644 index 4085b4ab9..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q17/explain.txt +++ /dev/null @@ -1,269 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (22) - : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet default.catalog_sales (10) - : : : : +- BroadcastExchange (20) - : : : : +- * Project (19) - : : : : +- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.date_dim (16) - : : : +- BroadcastExchange (27) - : : : +- * Project (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet default.date_dim (23) - : : +- ReusedExchange (30) - : +- BroadcastExchange (36) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.store (33) - +- BroadcastExchange (42) - +- * Filter (41) - +- * ColumnarToRow (40) - +- Scan parquet default.item (39) - - -(1) Scan parquet default.store_sales -Output [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6] -Condition : ((((isnotnull(ss_customer_sk#3) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#4)) - -(4) Scan parquet default.store_returns -Output [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] - -(6) Filter [codegen id : 1] -Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] -Condition : (((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) AND isnotnull(sr_returned_date_sk#7)) - -(7) BroadcastExchange -Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] -Arguments: HashedRelationBroadcastMode(List(input[2, bigint, false], input[1, bigint, false], input[3, bigint, false]),false), [id=#12] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11] -Input [11]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6, sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] - -(10) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16] - -(12) Filter [codegen id : 2] -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16] -Condition : ((isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) AND isnotnull(cs_sold_date_sk#13)) - -(13) BroadcastExchange -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint), cast(input[2, int, false] as bigint)),false), [id=#17] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cast(cs_bill_customer_sk#14 as bigint), cast(cs_item_sk#15 as bigint)] -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16] -Input [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16] - -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_quarter_name#19] -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)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#18, d_quarter_name#19] - -(18) Filter [codegen id : 3] -Input [2]: [d_date_sk#18, d_quarter_name#19] -Condition : ((isnotnull(d_quarter_name#19) AND (d_quarter_name#19 = 2001Q1)) AND isnotnull(d_date_sk#18)) - -(19) Project [codegen id : 3] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_quarter_name#19] - -(20) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] - -(21) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#18] -Join condition: None - -(22) Project [codegen id : 8] -Output [7]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16, d_date_sk#18] - -(23) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#21, d_quarter_name#22] - -(25) Filter [codegen id : 4] -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(26) Project [codegen id : 4] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_quarter_name#22] - -(27) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] - -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#7] -Right keys [1]: [cast(d_date_sk#21 as bigint)] -Join condition: None - -(29) Project [codegen id : 8] -Output [6]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16] -Input [8]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16, d_date_sk#21] - -(30) ReusedExchange [Reuses operator id: 27] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#24] -Join condition: None - -(32) Project [codegen id : 8] -Output [5]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16, d_date_sk#24] - -(33) Scan parquet default.store -Output [2]: [s_store_sk#25, s_state#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#25, s_state#26] - -(35) Filter [codegen id : 6] -Input [2]: [s_store_sk#25, s_state#26] -Condition : isnotnull(s_store_sk#25) - -(36) BroadcastExchange -Input [2]: [s_store_sk#25, s_state#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#25] -Join condition: None - -(38) Project [codegen id : 8] -Output [5]: [ss_item_sk#2, ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_state#26] -Input [7]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_store_sk#25, s_state#26] - -(39) Scan parquet default.item -Output [3]: [i_item_sk#28, i_item_id#29, i_item_desc#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(40) ColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#28, i_item_id#29, i_item_desc#30] - -(41) Filter [codegen id : 7] -Input [3]: [i_item_sk#28, i_item_id#29, i_item_desc#30] -Condition : isnotnull(i_item_sk#28) - -(42) BroadcastExchange -Input [3]: [i_item_sk#28, i_item_id#29, i_item_desc#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(43) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#28] -Join condition: None - -(44) Project [codegen id : 8] -Output [6]: [ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#29, i_item_desc#30] -Input [8]: [ss_item_sk#2, ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_sk#28, i_item_id#29, i_item_desc#30] - -(45) HashAggregate [codegen id : 8] -Input [6]: [ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#29, i_item_desc#30] -Keys [3]: [i_item_id#29, i_item_desc#30, s_state#26] -Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#11), partial_avg(cast(sr_return_quantity#11 as bigint)), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cast(cs_quantity#16 as bigint)), partial_stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [18]: [count#32, sum#33, count#34, n#35, avg#36, m2#37, count#38, sum#39, count#40, n#41, avg#42, m2#43, count#44, sum#45, count#46, n#47, avg#48, m2#49] -Results [21]: [i_item_id#29, i_item_desc#30, s_state#26, count#50, sum#51, count#52, n#53, avg#54, m2#55, count#56, sum#57, count#58, n#59, avg#60, m2#61, count#62, sum#63, count#64, n#65, avg#66, m2#67] - -(46) Exchange -Input [21]: [i_item_id#29, i_item_desc#30, s_state#26, count#50, sum#51, count#52, n#53, avg#54, m2#55, count#56, sum#57, count#58, n#59, avg#60, m2#61, count#62, sum#63, count#64, n#65, avg#66, m2#67] -Arguments: hashpartitioning(i_item_id#29, i_item_desc#30, s_state#26, 5), true, [id=#68] - -(47) HashAggregate [codegen id : 9] -Input [21]: [i_item_id#29, i_item_desc#30, s_state#26, count#50, sum#51, count#52, n#53, avg#54, m2#55, count#56, sum#57, count#58, n#59, avg#60, m2#61, count#62, sum#63, count#64, n#65, avg#66, m2#67] -Keys [3]: [i_item_id#29, i_item_desc#30, s_state#26] -Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#11), avg(cast(sr_return_quantity#11 as bigint)), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cast(cs_quantity#16 as bigint)), stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [9]: [count(ss_quantity#6)#69, avg(cast(ss_quantity#6 as bigint))#70, stddev_samp(cast(ss_quantity#6 as double))#71, count(sr_return_quantity#11)#72, avg(cast(sr_return_quantity#11 as bigint))#73, stddev_samp(cast(sr_return_quantity#11 as double))#74, count(cs_quantity#16)#75, avg(cast(cs_quantity#16 as bigint))#76, stddev_samp(cast(cs_quantity#16 as double))#77] -Results [15]: [i_item_id#29, i_item_desc#30, s_state#26, count(ss_quantity#6)#69 AS store_sales_quantitycount#78, avg(cast(ss_quantity#6 as bigint))#70 AS store_sales_quantityave#79, stddev_samp(cast(ss_quantity#6 as double))#71 AS store_sales_quantitystdev#80, (stddev_samp(cast(ss_quantity#6 as double))#71 / avg(cast(ss_quantity#6 as bigint))#70) AS store_sales_quantitycov#81, count(sr_return_quantity#11)#72 AS as_store_returns_quantitycount#82, avg(cast(sr_return_quantity#11 as bigint))#73 AS as_store_returns_quantityave#83, stddev_samp(cast(sr_return_quantity#11 as double))#74 AS as_store_returns_quantitystdev#84, (stddev_samp(cast(sr_return_quantity#11 as double))#74 / avg(cast(sr_return_quantity#11 as bigint))#73) AS store_returns_quantitycov#85, count(cs_quantity#16)#75 AS catalog_sales_quantitycount#86, avg(cast(cs_quantity#16 as bigint))#76 AS catalog_sales_quantityave#87, (stddev_samp(cast(cs_quantity#16 as double))#77 / avg(cast(cs_quantity#16 as bigint))#76) AS catalog_sales_quantitystdev#88, (stddev_samp(cast(cs_quantity#16 as double))#77 / avg(cast(cs_quantity#16 as bigint))#76) AS catalog_sales_quantitycov#89] - -(48) TakeOrderedAndProject -Input [15]: [i_item_id#29, i_item_desc#30, s_state#26, store_sales_quantitycount#78, store_sales_quantityave#79, store_sales_quantitystdev#80, store_sales_quantitycov#81, as_store_returns_quantitycount#82, as_store_returns_quantityave#83, as_store_returns_quantitystdev#84, store_returns_quantitycov#85, catalog_sales_quantitycount#86, catalog_sales_quantityave#87, catalog_sales_quantitystdev#88, catalog_sales_quantitycov#89] -Arguments: 100, [i_item_id#29 ASC NULLS FIRST, i_item_desc#30 ASC NULLS FIRST, s_state#26 ASC NULLS FIRST], [i_item_id#29, i_item_desc#30, s_state#26, store_sales_quantitycount#78, store_sales_quantityave#79, store_sales_quantitystdev#80, store_sales_quantitycov#81, as_store_returns_quantitycount#82, as_store_returns_quantityave#83, as_store_returns_quantitystdev#84, store_returns_quantitycov#85, catalog_sales_quantitycount#86, catalog_sales_quantityave#87, catalog_sales_quantitystdev#88, catalog_sales_quantitycov#89] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q17/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q17/simplified.txt deleted file mode 100644 index e9b95747c..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q17/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - InputAdapter - Exchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_sold_date_sk,cs_quantity] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_return_quantity,cs_sold_date_sk,cs_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_return_quantity,cs_sold_date_sk,cs_quantity] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_quarter_name,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_quarter_name,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q18/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q18/explain.txt deleted file mode 100644 index 3b213efa6..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q18/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Expand (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * Project (7) - : : : : : +- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.customer_demographics (4) - : : : : +- BroadcastExchange (15) - : : : : +- * Project (14) - : : : : +- * Filter (13) - : : : : +- * ColumnarToRow (12) - : : : : +- Scan parquet default.customer (11) - : : : +- BroadcastExchange (21) - : : : +- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet default.customer_demographics (18) - : : +- BroadcastExchange (27) - : : +- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.customer_address (24) - : +- BroadcastExchange (34) - : +- * Project (33) - : +- * Filter (32) - : +- * ColumnarToRow (31) - : +- Scan parquet default.date_dim (30) - +- BroadcastExchange (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet default.item (37) - - -(1) Scan parquet default.catalog_sales -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] - -(3) Filter [codegen id : 7] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) - -(4) Scan parquet default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] -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)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] - -(6) Filter [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] -Condition : ((((isnotnull(cd_gender#11) AND isnotnull(cd_education_status#12)) AND (cd_gender#11 = F)) AND (cd_education_status#12 = Unknown)) AND isnotnull(cd_demo_sk#10)) - -(7) Project [codegen id : 1] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] - -(8) BroadcastExchange -Input [2]: [cd_demo_sk#10, cd_dep_count#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] -Join condition: None - -(10) Project [codegen id : 7] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_demo_sk#10, cd_dep_count#13] - -(11) Scan parquet default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,6,8,9,12,2]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] - -(13) Filter [codegen id : 2] -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(14) Project [codegen id : 2] -Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] - -(15) BroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#15] -Join condition: None - -(17) Project [codegen id : 7] -Output [11]: [cs_sold_date_sk#1, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Input [13]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(18) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#21] - -(20) Filter [codegen id : 3] -Input [1]: [cd_demo_sk#21] -Condition : isnotnull(cd_demo_sk#21) - -(21) BroadcastExchange -Input [1]: [cd_demo_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#21] -Join condition: None - -(23) Project [codegen id : 7] -Output [10]: [cs_sold_date_sk#1, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_current_addr_sk#17, c_birth_year#19] -Input [12]: [cs_sold_date_sk#1, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#21] - -(24) Scan parquet default.customer_address -Output [4]: [ca_address_sk#23, ca_county#24, ca_state#25, ca_country#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_state, [MS,IN,ND,OK,NM,VA]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#23, ca_county#24, ca_state#25, ca_country#26] - -(26) Filter [codegen id : 4] -Input [4]: [ca_address_sk#23, ca_county#24, ca_state#25, ca_country#26] -Condition : (ca_state#25 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#23)) - -(27) BroadcastExchange -Input [4]: [ca_address_sk#23, ca_county#24, ca_state#25, ca_country#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] - -(28) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#23] -Join condition: None - -(29) Project [codegen id : 7] -Output [12]: [cs_sold_date_sk#1, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, ca_county#24, ca_state#25, ca_country#26] -Input [14]: [cs_sold_date_sk#1, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#23, ca_county#24, ca_state#25, ca_country#26] - -(30) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_year#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#28, d_year#29] - -(32) Filter [codegen id : 5] -Input [2]: [d_date_sk#28, d_year#29] -Condition : ((isnotnull(d_year#29) AND (d_year#29 = 1998)) AND isnotnull(d_date_sk#28)) - -(33) Project [codegen id : 5] -Output [1]: [d_date_sk#28] -Input [2]: [d_date_sk#28, d_year#29] - -(34) BroadcastExchange -Input [1]: [d_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] - -(35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#28] -Join condition: None - -(36) Project [codegen id : 7] -Output [11]: [cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, ca_county#24, ca_state#25, ca_country#26] -Input [13]: [cs_sold_date_sk#1, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, ca_county#24, ca_state#25, ca_country#26, d_date_sk#28] - -(37) Scan parquet default.item -Output [2]: [i_item_sk#31, i_item_id#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#31, i_item_id#32] - -(39) Filter [codegen id : 6] -Input [2]: [i_item_sk#31, i_item_id#32] -Condition : isnotnull(i_item_sk#31) - -(40) BroadcastExchange -Input [2]: [i_item_sk#31, i_item_id#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] - -(41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#31] -Join condition: None - -(42) Project [codegen id : 7] -Output [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, i_item_id#32, ca_country#26, ca_state#25, ca_county#24] -Input [13]: [cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, ca_county#24, ca_state#25, ca_country#26, i_item_sk#31, i_item_id#32] - -(43) Expand [codegen id : 7] -Input [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, i_item_id#32, ca_country#26, ca_state#25, ca_county#24] -Arguments: [List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, i_item_id#32, ca_country#26, ca_state#25, ca_county#24, 0), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, i_item_id#32, ca_country#26, ca_state#25, null, 1), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, i_item_id#32, ca_country#26, null, null, 3), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, i_item_id#32, null, null, null, 7), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, null, null, null, null, 15)], [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, i_item_id#34, ca_country#35, ca_state#36, ca_county#37, spark_grouping_id#38] - -(44) HashAggregate [codegen id : 7] -Input [12]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#19, i_item_id#34, ca_country#35, ca_state#36, ca_county#37, spark_grouping_id#38] -Keys [5]: [i_item_id#34, ca_country#35, ca_state#36, ca_county#37, spark_grouping_id#38] -Functions [7]: [partial_avg(cast(cs_quantity#5 as decimal(12,2))), partial_avg(cast(cs_list_price#6 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#8 as decimal(12,2))), partial_avg(cast(cs_sales_price#7 as decimal(12,2))), partial_avg(cast(cs_net_profit#9 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#13 as decimal(12,2)))] -Aggregate Attributes [14]: [sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52] -Results [19]: [i_item_id#34, ca_country#35, ca_state#36, ca_county#37, spark_grouping_id#38, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66] - -(45) Exchange -Input [19]: [i_item_id#34, ca_country#35, ca_state#36, ca_county#37, spark_grouping_id#38, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66] -Arguments: hashpartitioning(i_item_id#34, ca_country#35, ca_state#36, ca_county#37, spark_grouping_id#38, 5), true, [id=#67] - -(46) HashAggregate [codegen id : 8] -Input [19]: [i_item_id#34, ca_country#35, ca_state#36, ca_county#37, spark_grouping_id#38, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66] -Keys [5]: [i_item_id#34, ca_country#35, ca_state#36, ca_county#37, spark_grouping_id#38] -Functions [7]: [avg(cast(cs_quantity#5 as decimal(12,2))), avg(cast(cs_list_price#6 as decimal(12,2))), avg(cast(cs_coupon_amt#8 as decimal(12,2))), avg(cast(cs_sales_price#7 as decimal(12,2))), avg(cast(cs_net_profit#9 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#13 as decimal(12,2)))] -Aggregate Attributes [7]: [avg(cast(cs_quantity#5 as decimal(12,2)))#68, avg(cast(cs_list_price#6 as decimal(12,2)))#69, avg(cast(cs_coupon_amt#8 as decimal(12,2)))#70, avg(cast(cs_sales_price#7 as decimal(12,2)))#71, avg(cast(cs_net_profit#9 as decimal(12,2)))#72, avg(cast(c_birth_year#19 as decimal(12,2)))#73, avg(cast(cd_dep_count#13 as decimal(12,2)))#74] -Results [11]: [i_item_id#34, ca_country#35, ca_state#36, ca_county#37, avg(cast(cs_quantity#5 as decimal(12,2)))#68 AS agg1#75, avg(cast(cs_list_price#6 as decimal(12,2)))#69 AS agg2#76, avg(cast(cs_coupon_amt#8 as decimal(12,2)))#70 AS agg3#77, avg(cast(cs_sales_price#7 as decimal(12,2)))#71 AS agg4#78, avg(cast(cs_net_profit#9 as decimal(12,2)))#72 AS agg5#79, avg(cast(c_birth_year#19 as decimal(12,2)))#73 AS agg6#80, avg(cast(cd_dep_count#13 as decimal(12,2)))#74 AS agg7#81] - -(47) TakeOrderedAndProject -Input [11]: [i_item_id#34, ca_country#35, ca_state#36, ca_county#37, agg1#75, agg2#76, agg3#77, agg4#78, agg5#79, agg6#80, agg7#81] -Arguments: 100, [ca_country#35 ASC NULLS FIRST, ca_state#36 ASC NULLS FIRST, ca_county#37 ASC NULLS FIRST, i_item_id#34 ASC NULLS FIRST], [i_item_id#34, ca_country#35, ca_state#36, ca_county#37, agg1#75, agg2#76, agg3#77, agg4#78, agg5#79, agg6#80, agg7#81] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q18/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q18/simplified.txt deleted file mode 100644 index 0d4c8cb7f..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q18/simplified.txt +++ /dev/null @@ -1,69 +0,0 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [cd_demo_sk,cd_dep_count] - Filter [cd_gender,cd_education_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q19/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q19/explain.txt deleted file mode 100644 index cb75374a8..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q19/explain.txt +++ /dev/null @@ -1,221 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (17) - : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.date_dim (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet default.store_sales (5) - : : : +- BroadcastExchange (15) - : : : +- * Project (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.item (11) - : : +- BroadcastExchange (21) - : : +- * Filter (20) - : : +- * ColumnarToRow (19) - : : +- Scan parquet default.customer (18) - : +- BroadcastExchange (27) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- BroadcastExchange (33) - +- * Filter (32) - +- * ColumnarToRow (31) - +- Scan parquet default.store (30) - - -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] - -(3) Filter [codegen id : 6] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) - -(4) Project [codegen id : 6] -Output [1]: [d_date_sk#1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] - -(5) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#4, ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [5]: [ss_sold_date_sk#4, ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8] - -(7) Filter [codegen id : 1] -Input [5]: [ss_sold_date_sk#4, ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8] -Condition : (((isnotnull(ss_sold_date_sk#4) AND isnotnull(ss_item_sk#5)) AND isnotnull(ss_customer_sk#6)) AND isnotnull(ss_store_sk#7)) - -(8) BroadcastExchange -Input [5]: [ss_sold_date_sk#4, ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#9] - -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] -Join condition: None - -(10) Project [codegen id : 6] -Output [4]: [ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8] -Input [6]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8] - -(11) Scan parquet default.item -Output [6]: [i_item_sk#10, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14, i_manager_id#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [6]: [i_item_sk#10, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14, i_manager_id#15] - -(13) Filter [codegen id : 2] -Input [6]: [i_item_sk#10, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14, i_manager_id#15] -Condition : ((isnotnull(i_manager_id#15) AND (i_manager_id#15 = 8)) AND isnotnull(i_item_sk#10)) - -(14) Project [codegen id : 2] -Output [5]: [i_item_sk#10, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14] -Input [6]: [i_item_sk#10, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14, i_manager_id#15] - -(15) BroadcastExchange -Input [5]: [i_item_sk#10, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(16) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#10] -Join condition: None - -(17) Project [codegen id : 6] -Output [7]: [ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14] -Input [9]: [ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8, i_item_sk#10, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14] - -(18) Scan parquet default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] - -(20) Filter [codegen id : 3] -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(21) BroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] - -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#6] -Right keys [1]: [c_customer_sk#17] -Join condition: None - -(23) Project [codegen id : 6] -Output [7]: [ss_store_sk#7, ss_ext_sales_price#8, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14, c_current_addr_sk#18] -Input [9]: [ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14, c_customer_sk#17, c_current_addr_sk#18] - -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#20, ca_zip#21] - -(26) Filter [codegen id : 4] -Input [2]: [ca_address_sk#20, ca_zip#21] -Condition : (isnotnull(ca_address_sk#20) AND isnotnull(ca_zip#21)) - -(27) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_zip#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#20] -Join condition: None - -(29) Project [codegen id : 6] -Output [7]: [ss_store_sk#7, ss_ext_sales_price#8, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14, ca_zip#21] -Input [9]: [ss_store_sk#7, ss_ext_sales_price#8, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14, c_current_addr_sk#18, ca_address_sk#20, ca_zip#21] - -(30) Scan parquet default.store -Output [2]: [s_store_sk#23, s_zip#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 5] -Input [2]: [s_store_sk#23, s_zip#24] - -(32) Filter [codegen id : 5] -Input [2]: [s_store_sk#23, s_zip#24] -Condition : (isnotnull(s_zip#24) AND isnotnull(s_store_sk#23)) - -(33) BroadcastExchange -Input [2]: [s_store_sk#23, s_zip#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] - -(34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#23] -Join condition: NOT (substr(ca_zip#21, 1, 5) = substr(s_zip#24, 1, 5)) - -(35) Project [codegen id : 6] -Output [5]: [ss_ext_sales_price#8, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14] -Input [9]: [ss_store_sk#7, ss_ext_sales_price#8, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14, ca_zip#21, s_store_sk#23, s_zip#24] - -(36) HashAggregate [codegen id : 6] -Input [5]: [ss_ext_sales_price#8, i_brand_id#11, i_brand#12, i_manufact_id#13, i_manufact#14] -Keys [4]: [i_brand#12, i_brand_id#11, i_manufact_id#13, i_manufact#14] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#8))] -Aggregate Attributes [1]: [sum#26] -Results [5]: [i_brand#12, i_brand_id#11, i_manufact_id#13, i_manufact#14, sum#27] - -(37) Exchange -Input [5]: [i_brand#12, i_brand_id#11, i_manufact_id#13, i_manufact#14, sum#27] -Arguments: hashpartitioning(i_brand#12, i_brand_id#11, i_manufact_id#13, i_manufact#14, 5), true, [id=#28] - -(38) HashAggregate [codegen id : 7] -Input [5]: [i_brand#12, i_brand_id#11, i_manufact_id#13, i_manufact#14, sum#27] -Keys [4]: [i_brand#12, i_brand_id#11, i_manufact_id#13, i_manufact#14] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#8))#29] -Results [5]: [i_brand_id#11 AS brand_id#30, i_brand#12 AS brand#31, i_manufact_id#13, i_manufact#14, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#8))#29,17,2) AS ext_price#32] - -(39) TakeOrderedAndProject -Input [5]: [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32] -Arguments: 100, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#13 ASC NULLS FIRST, i_manufact#14 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q19/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q19/simplified.txt deleted file mode 100644 index 1bbbf35e4..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q19/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] - WholeStageCodegen (7) - HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - InputAdapter - Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - WholeStageCodegen (6) - HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] - Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - Filter [i_manager_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [ca_address_sk,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_zip] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Filter [s_zip,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_zip] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q2/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q2/explain.txt deleted file mode 100644 index be37495fd..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q2/explain.txt +++ /dev/null @@ -1,218 +0,0 @@ -== Physical Plan == -* Sort (39) -+- Exchange (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * HashAggregate (18) - : : +- Exchange (17) - : : +- * HashAggregate (16) - : : +- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- Union (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.web_sales (1) - : : : +- * Project (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet default.catalog_sales (5) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.date_dim (10) - : +- BroadcastExchange (23) - : +- * Project (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.date_dim (19) - +- BroadcastExchange (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * HashAggregate (27) - : +- ReusedExchange (26) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- * ColumnarToRow (29) - +- Scan parquet default.date_dim (28) - - -(1) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#1, ws_ext_sales_price#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [2]: [ws_sold_date_sk#1, ws_ext_sales_price#2] - -(3) Filter [codegen id : 1] -Input [2]: [ws_sold_date_sk#1, ws_ext_sales_price#2] -Condition : isnotnull(ws_sold_date_sk#1) - -(4) Project [codegen id : 1] -Output [2]: [ws_sold_date_sk#1 AS sold_date_sk#3, ws_ext_sales_price#2 AS sales_price#4] -Input [2]: [ws_sold_date_sk#1, ws_ext_sales_price#2] - -(5) Scan parquet default.catalog_sales -Output [2]: [cs_sold_date_sk#5, cs_ext_sales_price#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [2]: [cs_sold_date_sk#5, cs_ext_sales_price#6] - -(7) Filter [codegen id : 2] -Input [2]: [cs_sold_date_sk#5, cs_ext_sales_price#6] -Condition : isnotnull(cs_sold_date_sk#5) - -(8) Project [codegen id : 2] -Output [2]: [cs_sold_date_sk#5 AS sold_date_sk#7, cs_ext_sales_price#6 AS sales_price#8] -Input [2]: [cs_sold_date_sk#5, cs_ext_sales_price#6] - -(9) Union - -(10) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] - -(12) Filter [codegen id : 3] -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) - -(13) BroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sold_date_sk#3] -Right keys [1]: [d_date_sk#9] -Join condition: None - -(15) Project [codegen id : 4] -Output [3]: [sales_price#4, d_week_seq#10, d_day_name#11] -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#11] - -(16) HashAggregate [codegen id : 4] -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#11] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))] -Aggregate Attributes [7]: [sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Results [8]: [d_week_seq#10, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] - -(17) Exchange -Input [8]: [d_week_seq#10, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] -Arguments: hashpartitioning(d_week_seq#10, 5), true, [id=#27] - -(18) HashAggregate [codegen id : 12] -Input [8]: [d_week_seq#10, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#34] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#28,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#29,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#30,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#31,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#32,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#33,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#34,17,2) AS sat_sales#41] - -(19) Scan parquet default.date_dim -Output [2]: [d_week_seq#42, d_year#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 5] -Input [2]: [d_week_seq#42, d_year#43] - -(21) Filter [codegen id : 5] -Input [2]: [d_week_seq#42, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2001)) AND isnotnull(d_week_seq#42)) - -(22) Project [codegen id : 5] -Output [1]: [d_week_seq#42] -Input [2]: [d_week_seq#42, d_year#43] - -(23) BroadcastExchange -Input [1]: [d_week_seq#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] - -(24) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#42] -Join condition: None - -(25) Project [codegen id : 12] -Output [8]: [d_week_seq#10 AS d_week_seq1#45, sun_sales#35 AS sun_sales1#46, mon_sales#36 AS mon_sales1#47, tue_sales#37 AS tue_sales1#48, wed_sales#38 AS wed_sales1#49, thu_sales#39 AS thu_sales1#50, fri_sales#40 AS fri_sales1#51, sat_sales#41 AS sat_sales1#52] -Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#42] - -(26) ReusedExchange [Reuses operator id: 17] -Output [8]: [d_week_seq#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] - -(27) HashAggregate [codegen id : 11] -Input [8]: [d_week_seq#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,17,2) AS sat_sales#41] - -(28) Scan parquet default.date_dim -Output [2]: [d_week_seq#67, d_year#68] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 10] -Input [2]: [d_week_seq#67, d_year#68] - -(30) Filter [codegen id : 10] -Input [2]: [d_week_seq#67, d_year#68] -Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) - -(31) Project [codegen id : 10] -Output [1]: [d_week_seq#67] -Input [2]: [d_week_seq#67, d_year#68] - -(32) BroadcastExchange -Input [1]: [d_week_seq#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] - -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#67] -Join condition: None - -(34) Project [codegen id : 11] -Output [8]: [d_week_seq#10 AS d_week_seq2#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] -Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#67] - -(35) BroadcastExchange -Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] - -(36) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#70 - 53)] -Join condition: None - -(37) Project [codegen id : 12] -Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] -Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] - -(38) Exchange -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), true, [id=#86] - -(39) Sort [codegen id : 13] -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] -Arguments: [d_week_seq1#45 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q2/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q2/simplified.txt deleted file mode 100644 index f8028aa54..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q2/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (13) - Sort [d_week_seq1] - InputAdapter - Exchange [d_week_seq1] #1 - WholeStageCodegen (12) - Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - InputAdapter - Exchange [d_week_seq] #2 - WholeStageCodegen (4) - HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [sold_date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ws_sold_date_sk,ws_ext_sales_price] - Filter [ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_ext_sales_price] - WholeStageCodegen (2) - Project [cs_sold_date_sk,cs_ext_sales_price] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Filter [d_date_sk,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) - Project [d_week_seq] - Filter [d_year,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_week_seq,d_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - InputAdapter - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - Project [d_week_seq] - Filter [d_year,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_week_seq,d_year] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q20/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q20/explain.txt deleted file mode 100644 index 4234fba2b..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q20/explain.txt +++ /dev/null @@ -1,137 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * Sort (21) - +- Exchange (20) - +- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * BroadcastHashJoin Inner BuildRight (15) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.item (4) - +- BroadcastExchange (14) - +- * Project (13) - +- * Filter (12) - +- * ColumnarToRow (11) - +- Scan parquet default.date_dim (10) - - -(1) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] - -(3) Filter [codegen id : 3] -Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) - -(4) Scan parquet default.item -Output [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] - -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Condition : (i_category#9 IN (Sports,Books,Home) AND isnotnull(i_item_sk#4)) - -(7) BroadcastExchange -Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#4] -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Input [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] - -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] - -(12) Filter [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10644)) AND (d_date#12 <= 10674)) AND isnotnull(d_date_sk#11)) - -(13) Project [codegen id : 2] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_date#12] - -(14) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#11] -Join condition: None - -(16) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Input [8]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9, d_date_sk#11] - -(17) HashAggregate [codegen id : 3] -Input [6]: [cs_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Keys [5]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#14] -Results [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#15] - -(18) Exchange -Input [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#15] -Arguments: hashpartitioning(i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, 5), true, [id=#16] - -(19) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#15] -Keys [5]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#3))#17] -Results [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#17,17,2) AS _w0#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#17,17,2) AS _w1#20, i_item_id#5] - -(20) Exchange -Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5] -Arguments: hashpartitioning(i_class#8, 5), true, [id=#21] - -(21) Sort [codegen id : 5] -Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5] -Arguments: [i_class#8 ASC NULLS FIRST], false, 0 - -(22) Window -Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5] -Arguments: [sum(_w1#20) windowspecdefinition(i_class#8, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#22], [i_class#8] - -(23) Project [codegen id : 6] -Output [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#19) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#22)), DecimalType(38,17), true) AS revenueratio#23, i_item_id#5] -Input [9]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5, _we0#22] - -(24) TakeOrderedAndProject -Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, revenueratio#23, i_item_id#5] -Arguments: 100, [i_category#9 ASC NULLS FIRST, i_class#8 ASC NULLS FIRST, i_item_id#5 ASC NULLS FIRST, i_item_desc#6 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, revenueratio#23] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q20/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q20/simplified.txt deleted file mode 100644 index 6259c1d53..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q20/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w1,i_class] - WholeStageCodegen (5) - Sort [i_class] - InputAdapter - Exchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,_w1,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q21/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q21/explain.txt deleted file mode 100644 index 788d1affd..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q21/explain.txt +++ /dev/null @@ -1,155 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (27) -+- * Filter (26) - +- * HashAggregate (25) - +- Exchange (24) - +- * HashAggregate (23) - +- * Project (22) - +- * BroadcastHashJoin Inner BuildRight (21) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.inventory (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.warehouse (4) - : +- BroadcastExchange (14) - : +- * Project (13) - : +- * Filter (12) - : +- * ColumnarToRow (11) - : +- Scan parquet default.item (10) - +- BroadcastExchange (20) - +- * Filter (19) - +- * ColumnarToRow (18) - +- Scan parquet default.date_dim (17) - - -(1) Scan parquet default.inventory -Output [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk), IsNotNull(inv_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Condition : ((isnotnull(inv_warehouse_sk#3) AND isnotnull(inv_item_sk#2)) AND isnotnull(inv_date_sk#1)) - -(4) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#5, w_warehouse_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#5, w_warehouse_name#6] - -(6) Filter [codegen id : 1] -Input [2]: [w_warehouse_sk#5, w_warehouse_name#6] -Condition : isnotnull(w_warehouse_sk#5) - -(7) BroadcastExchange -Input [2]: [w_warehouse_sk#5, w_warehouse_name#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#5] -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_date_sk#1, inv_item_sk#2, inv_quantity_on_hand#4, w_warehouse_name#6] -Input [6]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4, w_warehouse_sk#5, w_warehouse_name#6] - -(10) Scan parquet default.item -Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] - -(12) Filter [codegen id : 2] -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) - -(13) Project [codegen id : 2] -Output [2]: [i_item_sk#8, i_item_id#9] -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] - -(14) BroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#2] -Right keys [1]: [i_item_sk#8] -Join condition: None - -(16) Project [codegen id : 4] -Output [4]: [inv_date_sk#1, inv_quantity_on_hand#4, w_warehouse_name#6, i_item_id#9] -Input [6]: [inv_date_sk#1, inv_item_sk#2, inv_quantity_on_hand#4, w_warehouse_name#6, i_item_sk#8, i_item_id#9] - -(17) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_date#13] - -(19) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10997)) AND (d_date#13 <= 11057)) AND isnotnull(d_date_sk#12)) - -(20) BroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#1] -Right keys [1]: [d_date_sk#12] -Join condition: None - -(22) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#4, w_warehouse_name#6, i_item_id#9, d_date#13] -Input [6]: [inv_date_sk#1, inv_quantity_on_hand#4, w_warehouse_name#6, i_item_id#9, d_date_sk#12, d_date#13] - -(23) HashAggregate [codegen id : 4] -Input [4]: [inv_quantity_on_hand#4, w_warehouse_name#6, i_item_id#9, d_date#13] -Keys [2]: [w_warehouse_name#6, i_item_id#9] -Functions [2]: [partial_sum(cast(CASE WHEN (d_date#13 < 11027) THEN inv_quantity_on_hand#4 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (d_date#13 >= 11027) THEN inv_quantity_on_hand#4 ELSE 0 END as bigint))] -Aggregate Attributes [2]: [sum#15, sum#16] -Results [4]: [w_warehouse_name#6, i_item_id#9, sum#17, sum#18] - -(24) Exchange -Input [4]: [w_warehouse_name#6, i_item_id#9, sum#17, sum#18] -Arguments: hashpartitioning(w_warehouse_name#6, i_item_id#9, 5), true, [id=#19] - -(25) HashAggregate [codegen id : 5] -Input [4]: [w_warehouse_name#6, i_item_id#9, sum#17, sum#18] -Keys [2]: [w_warehouse_name#6, i_item_id#9] -Functions [2]: [sum(cast(CASE WHEN (d_date#13 < 11027) THEN inv_quantity_on_hand#4 ELSE 0 END as bigint)), sum(cast(CASE WHEN (d_date#13 >= 11027) THEN inv_quantity_on_hand#4 ELSE 0 END as bigint))] -Aggregate Attributes [2]: [sum(cast(CASE WHEN (d_date#13 < 11027) THEN inv_quantity_on_hand#4 ELSE 0 END as bigint))#20, sum(cast(CASE WHEN (d_date#13 >= 11027) THEN inv_quantity_on_hand#4 ELSE 0 END as bigint))#21] -Results [4]: [w_warehouse_name#6, i_item_id#9, sum(cast(CASE WHEN (d_date#13 < 11027) THEN inv_quantity_on_hand#4 ELSE 0 END as bigint))#20 AS inv_before#22, sum(cast(CASE WHEN (d_date#13 >= 11027) THEN inv_quantity_on_hand#4 ELSE 0 END as bigint))#21 AS inv_after#23] - -(26) Filter [codegen id : 5] -Input [4]: [w_warehouse_name#6, i_item_id#9, inv_before#22, inv_after#23] -Condition : ((CASE WHEN (inv_before#22 > 0) THEN (cast(inv_after#23 as double) / cast(inv_before#22 as double)) ELSE null END >= 0.666667) AND (CASE WHEN (inv_before#22 > 0) THEN (cast(inv_after#23 as double) / cast(inv_before#22 as double)) ELSE null END <= 1.5)) - -(27) TakeOrderedAndProject -Input [4]: [w_warehouse_name#6, i_item_id#9, inv_before#22, inv_after#23] -Arguments: 100, [w_warehouse_name#6 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#6, i_item_id#9, inv_before#22, inv_after#23] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q21/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q21/simplified.txt deleted file mode 100644 index 9b5483bd7..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q21/simplified.txt +++ /dev/null @@ -1,40 +0,0 @@ -TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - WholeStageCodegen (5) - Filter [inv_before,inv_after] - HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(cast(CASE WHEN (d_date < 11027) THEN inv_quantity_on_hand ELSE 0 END as bigint)),sum(cast(CASE WHEN (d_date >= 11027) THEN inv_quantity_on_hand ELSE 0 END as bigint)),inv_before,inv_after,sum,sum] - InputAdapter - Exchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_date_sk,inv_quantity_on_hand,w_warehouse_name,i_item_id] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_warehouse_sk,inv_item_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [i_item_sk,i_item_id] - Filter [i_current_price,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q22/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q22/explain.txt deleted file mode 100644 index 6aae0b0c8..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q22/explain.txt +++ /dev/null @@ -1,155 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (27) -+- * HashAggregate (26) - +- Exchange (25) - +- * HashAggregate (24) - +- * Expand (23) - +- * Project (22) - +- * BroadcastHashJoin Inner BuildRight (21) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.inventory (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.date_dim (4) - : +- BroadcastExchange (14) - : +- * Filter (13) - : +- * ColumnarToRow (12) - : +- Scan parquet default.item (11) - +- BroadcastExchange (20) - +- * Filter (19) - +- * ColumnarToRow (18) - +- Scan parquet default.warehouse (17) - - -(1) Scan parquet default.inventory -Output [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_date_sk), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Condition : ((isnotnull(inv_date_sk#1) AND isnotnull(inv_item_sk#2)) AND isnotnull(inv_warehouse_sk#3)) - -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_month_seq#6] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_month_seq#6] - -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#1] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(10) Project [codegen id : 4] -Output [3]: [inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Input [5]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4, d_date_sk#5] - -(11) Scan parquet default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] - -(13) Filter [codegen id : 2] -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(14) BroadcastExchange -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#2] -Right keys [1]: [i_item_sk#8] -Join condition: None - -(16) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#3, inv_quantity_on_hand#4, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Input [8]: [inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] - -(17) Scan parquet default.warehouse -Output [1]: [w_warehouse_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#14] - -(19) Filter [codegen id : 3] -Input [1]: [w_warehouse_sk#14] -Condition : isnotnull(w_warehouse_sk#14) - -(20) BroadcastExchange -Input [1]: [w_warehouse_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#14] -Join condition: None - -(22) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#4, i_product_name#12, i_brand#9, i_class#10, i_category#11] -Input [7]: [inv_warehouse_sk#3, inv_quantity_on_hand#4, i_brand#9, i_class#10, i_category#11, i_product_name#12, w_warehouse_sk#14] - -(23) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#4, i_product_name#12, i_brand#9, i_class#10, i_category#11] -Arguments: [List(inv_quantity_on_hand#4, i_product_name#12, i_brand#9, i_class#10, i_category#11, 0), List(inv_quantity_on_hand#4, i_product_name#12, i_brand#9, i_class#10, null, 1), List(inv_quantity_on_hand#4, i_product_name#12, i_brand#9, null, null, 3), List(inv_quantity_on_hand#4, i_product_name#12, null, null, null, 7), List(inv_quantity_on_hand#4, null, null, null, null, 15)], [inv_quantity_on_hand#4, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] - -(24) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#4, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [partial_avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [sum#21, count#22] -Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(25) Exchange -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), true, [id=#25] - -(26) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS qoh#27] - -(27) TakeOrderedAndProject -Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#27] -Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#27] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q22/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q22/simplified.txt deleted file mode 100644 index 233babdf3..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q22/simplified.txt +++ /dev/null @@ -1,40 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] - InputAdapter - Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23a/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23a/explain.txt deleted file mode 100644 index 15ae5bfe2..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23a/explain.txt +++ /dev/null @@ -1,537 +0,0 @@ -== Physical Plan == -* HashAggregate (71) -+- Exchange (70) - +- * HashAggregate (69) - +- Union (68) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (44) - : : +- * BroadcastHashJoin LeftSemi BuildRight (43) - : : :- * Project (27) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_sales (1) - : : : +- BroadcastExchange (25) - : : : +- * Project (24) - : : : +- * Filter (23) - : : : +- * HashAggregate (22) - : : : +- Exchange (21) - : : : +- * HashAggregate (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (17) - : : : +- * Filter (16) - : : : +- * ColumnarToRow (15) - : : : +- Scan parquet default.item (14) - : : +- BroadcastExchange (42) - : : +- * Project (41) - : : +- * Filter (40) - : : +- * HashAggregate (39) - : : +- Exchange (38) - : : +- * HashAggregate (37) - : : +- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet default.store_sales (28) - : : +- BroadcastExchange (34) - : : +- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet default.customer (31) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.date_dim (45) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin LeftSemi BuildRight (63) - : :- * Project (57) - : : +- * BroadcastHashJoin LeftSemi BuildRight (56) - : : :- * Filter (54) - : : : +- * ColumnarToRow (53) - : : : +- Scan parquet default.web_sales (52) - : : +- ReusedExchange (55) - : +- BroadcastExchange (62) - : +- * Project (61) - : +- * Filter (60) - : +- * HashAggregate (59) - : +- ReusedExchange (58) - +- ReusedExchange (65) - - -(1) Scan parquet default.catalog_sales -Output [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 9] -Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] - -(3) Filter [codegen id : 9] -Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -Condition : isnotnull(cs_sold_date_sk#1) - -(4) Scan parquet default.store_sales -Output [2]: [ss_sold_date_sk#6, ss_item_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 3] -Input [2]: [ss_sold_date_sk#6, ss_item_sk#7] - -(6) Filter [codegen id : 3] -Input [2]: [ss_sold_date_sk#6, ss_item_sk#7] -Condition : (isnotnull(ss_sold_date_sk#6) AND isnotnull(ss_item_sk#7)) - -(7) Scan parquet default.date_dim -Output [3]: [d_date_sk#8, d_date#9, d_year#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#8, d_date#9, d_year#10] - -(9) Filter [codegen id : 1] -Input [3]: [d_date_sk#8, d_date#9, d_year#10] -Condition : (d_year#10 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#8)) - -(10) Project [codegen id : 1] -Output [2]: [d_date_sk#8, d_date#9] -Input [3]: [d_date_sk#8, d_date#9, d_year#10] - -(11) BroadcastExchange -Input [2]: [d_date_sk#8, d_date#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(13) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#9] -Input [4]: [ss_sold_date_sk#6, ss_item_sk#7, d_date_sk#8, d_date#9] - -(14) Scan parquet default.item -Output [2]: [i_item_sk#12, i_item_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) ColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] - -(16) Filter [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) - -(17) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(18) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(19) Project [codegen id : 3] -Output [3]: [d_date#9, i_item_sk#12, i_item_desc#13] -Input [4]: [ss_item_sk#7, d_date#9, i_item_sk#12, i_item_desc#13] - -(20) HashAggregate [codegen id : 3] -Input [3]: [d_date#9, i_item_sk#12, i_item_desc#13] -Keys [3]: [substr(i_item_desc#13, 1, 30) AS substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#16] -Results [4]: [substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9, count#17] - -(21) Exchange -Input [4]: [substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9, count#17] -Arguments: hashpartitioning(substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9, 5), true, [id=#18] - -(22) HashAggregate [codegen id : 4] -Input [4]: [substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9, count#17] -Keys [3]: [substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#19] -Results [2]: [i_item_sk#12 AS item_sk#20, count(1)#19 AS count(1)#21] - -(23) Filter [codegen id : 4] -Input [2]: [item_sk#20, count(1)#21] -Condition : (count(1)#21 > 4) - -(24) Project [codegen id : 4] -Output [1]: [item_sk#20] -Input [2]: [item_sk#20, count(1)#21] - -(25) BroadcastExchange -Input [1]: [item_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] - -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [item_sk#20] -Join condition: None - -(27) Project [codegen id : 9] -Output [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] -Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] - -(28) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 6] -Input [3]: [ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] - -(30) Filter [codegen id : 6] -Input [3]: [ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] -Condition : isnotnull(ss_customer_sk#23) - -(31) Scan parquet default.customer -Output [1]: [c_customer_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(32) ColumnarToRow [codegen id : 5] -Input [1]: [c_customer_sk#26] - -(33) Filter [codegen id : 5] -Input [1]: [c_customer_sk#26] -Condition : isnotnull(c_customer_sk#26) - -(34) BroadcastExchange -Input [1]: [c_customer_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] - -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#23] -Right keys [1]: [c_customer_sk#26] -Join condition: None - -(36) Project [codegen id : 6] -Output [3]: [ss_quantity#24, ss_sales_price#25, c_customer_sk#26] -Input [4]: [ss_customer_sk#23, ss_quantity#24, ss_sales_price#25, c_customer_sk#26] - -(37) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#24, ss_sales_price#25, c_customer_sk#26] -Keys [1]: [c_customer_sk#26] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#28, isEmpty#29] -Results [3]: [c_customer_sk#26, sum#30, isEmpty#31] - -(38) Exchange -Input [3]: [c_customer_sk#26, sum#30, isEmpty#31] -Arguments: hashpartitioning(c_customer_sk#26, 5), true, [id=#32] - -(39) HashAggregate [codegen id : 7] -Input [3]: [c_customer_sk#26, sum#30, isEmpty#31] -Keys [1]: [c_customer_sk#26] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#33] -Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#33 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34] - -(40) Filter [codegen id : 7] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#35, [id=#36] as decimal(32,6)))), DecimalType(38,8), true))) - -(41) Project [codegen id : 7] -Output [1]: [c_customer_sk#26] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34] - -(42) BroadcastExchange -Input [1]: [c_customer_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#37] - -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#26] -Join condition: None - -(44) Project [codegen id : 9] -Output [3]: [cs_sold_date_sk#1, cs_quantity#4, cs_list_price#5] -Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] - -(45) Scan parquet default.date_dim -Output [3]: [d_date_sk#8, d_year#10, d_moy#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 8] -Input [3]: [d_date_sk#8, d_year#10, d_moy#38] - -(47) Filter [codegen id : 8] -Input [3]: [d_date_sk#8, d_year#10, d_moy#38] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#38)) AND (d_year#10 = 2000)) AND (d_moy#38 = 2)) AND isnotnull(d_date_sk#8)) - -(48) Project [codegen id : 8] -Output [1]: [d_date_sk#8] -Input [3]: [d_date_sk#8, d_year#10, d_moy#38] - -(49) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] - -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(51) Project [codegen id : 9] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true) AS sales#40] -Input [4]: [cs_sold_date_sk#1, cs_quantity#4, cs_list_price#5, d_date_sk#8] - -(52) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(53) ColumnarToRow [codegen id : 18] -Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] - -(54) Filter [codegen id : 18] -Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -Condition : isnotnull(ws_sold_date_sk#41) - -(55) ReusedExchange [Reuses operator id: 25] -Output [1]: [item_sk#20] - -(56) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_item_sk#42] -Right keys [1]: [item_sk#20] -Join condition: None - -(57) Project [codegen id : 18] -Output [4]: [ws_sold_date_sk#41, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] - -(58) ReusedExchange [Reuses operator id: 38] -Output [3]: [c_customer_sk#26, sum#46, isEmpty#47] - -(59) HashAggregate [codegen id : 16] -Input [3]: [c_customer_sk#26, sum#46, isEmpty#47] -Keys [1]: [c_customer_sk#26] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#48] -Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#48 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#49] - -(60) Filter [codegen id : 16] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#49] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#49) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#49 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#35, [id=#36] as decimal(32,6)))), DecimalType(38,8), true))) - -(61) Project [codegen id : 16] -Output [1]: [c_customer_sk#26] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#49] - -(62) BroadcastExchange -Input [1]: [c_customer_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] - -(63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_customer_sk#43] -Right keys [1]: [c_customer_sk#26] -Join condition: None - -(64) Project [codegen id : 18] -Output [3]: [ws_sold_date_sk#41, ws_quantity#44, ws_list_price#45] -Input [4]: [ws_sold_date_sk#41, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] - -(65) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#8] - -(66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#41] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(67) Project [codegen id : 18] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#44 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#45 as decimal(12,2)))), DecimalType(18,2), true) AS sales#51] -Input [4]: [ws_sold_date_sk#41, ws_quantity#44, ws_list_price#45, d_date_sk#8] - -(68) Union - -(69) HashAggregate [codegen id : 19] -Input [1]: [sales#40] -Keys: [] -Functions [1]: [partial_sum(sales#40)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [2]: [sum#54, isEmpty#55] - -(70) Exchange -Input [2]: [sum#54, isEmpty#55] -Arguments: SinglePartition, true, [id=#56] - -(71) HashAggregate [codegen id : 20] -Input [2]: [sum#54, isEmpty#55] -Keys: [] -Functions [1]: [sum(sales#40)] -Aggregate Attributes [1]: [sum(sales#40)#57] -Results [1]: [sum(sales#40)#57 AS sum(sales)#58] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* HashAggregate (93) -+- Exchange (92) - +- * HashAggregate (91) - +- * HashAggregate (90) - +- Exchange (89) - +- * HashAggregate (88) - +- * Project (87) - +- * BroadcastHashJoin Inner BuildRight (86) - :- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * Filter (74) - : : +- * ColumnarToRow (73) - : : +- Scan parquet default.store_sales (72) - : +- BroadcastExchange (78) - : +- * Filter (77) - : +- * ColumnarToRow (76) - : +- Scan parquet default.customer (75) - +- BroadcastExchange (85) - +- * Project (84) - +- * Filter (83) - +- * ColumnarToRow (82) - +- Scan parquet default.date_dim (81) - - -(72) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#6, ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(73) ColumnarToRow [codegen id : 3] -Input [4]: [ss_sold_date_sk#6, ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] - -(74) Filter [codegen id : 3] -Input [4]: [ss_sold_date_sk#6, ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] -Condition : (isnotnull(ss_customer_sk#23) AND isnotnull(ss_sold_date_sk#6)) - -(75) Scan parquet default.customer -Output [1]: [c_customer_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(76) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#26] - -(77) Filter [codegen id : 1] -Input [1]: [c_customer_sk#26] -Condition : isnotnull(c_customer_sk#26) - -(78) BroadcastExchange -Input [1]: [c_customer_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] - -(79) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#23] -Right keys [1]: [c_customer_sk#26] -Join condition: None - -(80) Project [codegen id : 3] -Output [4]: [ss_sold_date_sk#6, ss_quantity#24, ss_sales_price#25, c_customer_sk#26] -Input [5]: [ss_sold_date_sk#6, ss_customer_sk#23, ss_quantity#24, ss_sales_price#25, c_customer_sk#26] - -(81) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_year#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(82) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#8, d_year#10] - -(83) Filter [codegen id : 2] -Input [2]: [d_date_sk#8, d_year#10] -Condition : (d_year#10 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#8)) - -(84) Project [codegen id : 2] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_year#10] - -(85) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] - -(86) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(87) Project [codegen id : 3] -Output [3]: [ss_quantity#24, ss_sales_price#25, c_customer_sk#26] -Input [5]: [ss_sold_date_sk#6, ss_quantity#24, ss_sales_price#25, c_customer_sk#26, d_date_sk#8] - -(88) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#24, ss_sales_price#25, c_customer_sk#26] -Keys [1]: [c_customer_sk#26] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#61, isEmpty#62] -Results [3]: [c_customer_sk#26, sum#63, isEmpty#64] - -(89) Exchange -Input [3]: [c_customer_sk#26, sum#63, isEmpty#64] -Arguments: hashpartitioning(c_customer_sk#26, 5), true, [id=#65] - -(90) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#26, sum#63, isEmpty#64] -Keys [1]: [c_customer_sk#26] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#66] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#66 AS csales#67] - -(91) HashAggregate [codegen id : 4] -Input [1]: [csales#67] -Keys: [] -Functions [1]: [partial_max(csales#67)] -Aggregate Attributes [1]: [max#68] -Results [1]: [max#69] - -(92) Exchange -Input [1]: [max#69] -Arguments: SinglePartition, true, [id=#70] - -(93) HashAggregate [codegen id : 5] -Input [1]: [max#69] -Keys: [] -Functions [1]: [max(csales#67)] -Aggregate Attributes [1]: [max(csales#67)#71] -Results [1]: [max(csales#67)#71 AS tpcds_cmax#72] - -Subquery:2 Hosting operator id = 60 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23a/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23a/simplified.txt deleted file mode 100644 index aebe2bd3e..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23a/simplified.txt +++ /dev/null @@ -1,142 +0,0 @@ -WholeStageCodegen (20) - HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] - InputAdapter - Exchange #1 - WholeStageCodegen (19) - HashAggregate [sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (9) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_quantity,cs_list_price] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_quantity,cs_list_price] - BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (4) - Project [item_sk] - Filter [count(1)] - HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] - InputAdapter - Exchange [substr(i_item_desc, 1, 30),i_item_sk,d_date] #3 - WholeStageCodegen (3) - HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - Subquery #1 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - Exchange #9 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),csales,sum,isEmpty] - InputAdapter - Exchange [c_customer_sk] #10 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - InputAdapter - Exchange [c_customer_sk] #7 - WholeStageCodegen (6) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (18) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_quantity,ws_list_price] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_sold_date_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] - BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] - InputAdapter - ReusedExchange [item_sk] #2 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (16) - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #7 - InputAdapter - ReusedExchange [d_date_sk] #13 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23b/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23b/explain.txt deleted file mode 100644 index b5213786c..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23b/explain.txt +++ /dev/null @@ -1,689 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (97) -+- Union (96) - :- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (55) - : : +- * BroadcastHashJoin Inner BuildRight (54) - : : :- * BroadcastHashJoin LeftSemi BuildRight (43) - : : : :- * Project (27) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- BroadcastExchange (25) - : : : : +- * Project (24) - : : : : +- * Filter (23) - : : : : +- * HashAggregate (22) - : : : : +- Exchange (21) - : : : : +- * HashAggregate (20) - : : : : +- * Project (19) - : : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : : :- * Project (13) - : : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : : :- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet default.store_sales (4) - : : : : : +- BroadcastExchange (11) - : : : : : +- * Project (10) - : : : : : +- * Filter (9) - : : : : : +- * ColumnarToRow (8) - : : : : : +- Scan parquet default.date_dim (7) - : : : : +- BroadcastExchange (17) - : : : : +- * Filter (16) - : : : : +- * ColumnarToRow (15) - : : : : +- Scan parquet default.item (14) - : : : +- BroadcastExchange (42) - : : : +- * Project (41) - : : : +- * Filter (40) - : : : +- * HashAggregate (39) - : : : +- Exchange (38) - : : : +- * HashAggregate (37) - : : : +- * Project (36) - : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : :- * Filter (30) - : : : : +- * ColumnarToRow (29) - : : : : +- Scan parquet default.store_sales (28) - : : : +- BroadcastExchange (34) - : : : +- * Filter (33) - : : : +- * ColumnarToRow (32) - : : : +- Scan parquet default.customer (31) - : : +- BroadcastExchange (53) - : : +- * BroadcastHashJoin LeftSemi BuildRight (52) - : : :- * Filter (46) - : : : +- * ColumnarToRow (45) - : : : +- Scan parquet default.customer (44) - : : +- BroadcastExchange (51) - : : +- * Project (50) - : : +- * Filter (49) - : : +- * HashAggregate (48) - : : +- ReusedExchange (47) - : +- BroadcastExchange (60) - : +- * Project (59) - : +- * Filter (58) - : +- * ColumnarToRow (57) - : +- Scan parquet default.date_dim (56) - +- * HashAggregate (95) - +- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * BroadcastHashJoin Inner BuildRight (91) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * BroadcastHashJoin LeftSemi BuildRight (77) - : : :- * Project (71) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (70) - : : : :- * Filter (68) - : : : : +- * ColumnarToRow (67) - : : : : +- Scan parquet default.web_sales (66) - : : : +- ReusedExchange (69) - : : +- BroadcastExchange (76) - : : +- * Project (75) - : : +- * Filter (74) - : : +- * HashAggregate (73) - : : +- ReusedExchange (72) - : +- BroadcastExchange (87) - : +- * BroadcastHashJoin LeftSemi BuildRight (86) - : :- * Filter (80) - : : +- * ColumnarToRow (79) - : : +- Scan parquet default.customer (78) - : +- BroadcastExchange (85) - : +- * Project (84) - : +- * Filter (83) - : +- * HashAggregate (82) - : +- ReusedExchange (81) - +- ReusedExchange (90) - - -(1) Scan parquet default.catalog_sales -Output [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 13] -Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] - -(3) Filter [codegen id : 13] -Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -Condition : (isnotnull(cs_bill_customer_sk#2) AND isnotnull(cs_sold_date_sk#1)) - -(4) Scan parquet default.store_sales -Output [2]: [ss_sold_date_sk#6, ss_item_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 3] -Input [2]: [ss_sold_date_sk#6, ss_item_sk#7] - -(6) Filter [codegen id : 3] -Input [2]: [ss_sold_date_sk#6, ss_item_sk#7] -Condition : (isnotnull(ss_sold_date_sk#6) AND isnotnull(ss_item_sk#7)) - -(7) Scan parquet default.date_dim -Output [3]: [d_date_sk#8, d_date#9, d_year#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#8, d_date#9, d_year#10] - -(9) Filter [codegen id : 1] -Input [3]: [d_date_sk#8, d_date#9, d_year#10] -Condition : (d_year#10 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#8)) - -(10) Project [codegen id : 1] -Output [2]: [d_date_sk#8, d_date#9] -Input [3]: [d_date_sk#8, d_date#9, d_year#10] - -(11) BroadcastExchange -Input [2]: [d_date_sk#8, d_date#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(13) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#9] -Input [4]: [ss_sold_date_sk#6, ss_item_sk#7, d_date_sk#8, d_date#9] - -(14) Scan parquet default.item -Output [2]: [i_item_sk#12, i_item_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) ColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] - -(16) Filter [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) - -(17) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(18) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(19) Project [codegen id : 3] -Output [3]: [d_date#9, i_item_sk#12, i_item_desc#13] -Input [4]: [ss_item_sk#7, d_date#9, i_item_sk#12, i_item_desc#13] - -(20) HashAggregate [codegen id : 3] -Input [3]: [d_date#9, i_item_sk#12, i_item_desc#13] -Keys [3]: [substr(i_item_desc#13, 1, 30) AS substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#16] -Results [4]: [substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9, count#17] - -(21) Exchange -Input [4]: [substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9, count#17] -Arguments: hashpartitioning(substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9, 5), true, [id=#18] - -(22) HashAggregate [codegen id : 4] -Input [4]: [substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9, count#17] -Keys [3]: [substr(i_item_desc#13, 1, 30)#15, i_item_sk#12, d_date#9] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#19] -Results [2]: [i_item_sk#12 AS item_sk#20, count(1)#19 AS count(1)#21] - -(23) Filter [codegen id : 4] -Input [2]: [item_sk#20, count(1)#21] -Condition : (count(1)#21 > 4) - -(24) Project [codegen id : 4] -Output [1]: [item_sk#20] -Input [2]: [item_sk#20, count(1)#21] - -(25) BroadcastExchange -Input [1]: [item_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] - -(26) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [item_sk#20] -Join condition: None - -(27) Project [codegen id : 13] -Output [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] -Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] - -(28) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 6] -Input [3]: [ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] - -(30) Filter [codegen id : 6] -Input [3]: [ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] -Condition : isnotnull(ss_customer_sk#23) - -(31) Scan parquet default.customer -Output [1]: [c_customer_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(32) ColumnarToRow [codegen id : 5] -Input [1]: [c_customer_sk#26] - -(33) Filter [codegen id : 5] -Input [1]: [c_customer_sk#26] -Condition : isnotnull(c_customer_sk#26) - -(34) BroadcastExchange -Input [1]: [c_customer_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] - -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#23] -Right keys [1]: [c_customer_sk#26] -Join condition: None - -(36) Project [codegen id : 6] -Output [3]: [ss_quantity#24, ss_sales_price#25, c_customer_sk#26] -Input [4]: [ss_customer_sk#23, ss_quantity#24, ss_sales_price#25, c_customer_sk#26] - -(37) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#24, ss_sales_price#25, c_customer_sk#26] -Keys [1]: [c_customer_sk#26] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#28, isEmpty#29] -Results [3]: [c_customer_sk#26, sum#30, isEmpty#31] - -(38) Exchange -Input [3]: [c_customer_sk#26, sum#30, isEmpty#31] -Arguments: hashpartitioning(c_customer_sk#26, 5), true, [id=#32] - -(39) HashAggregate [codegen id : 7] -Input [3]: [c_customer_sk#26, sum#30, isEmpty#31] -Keys [1]: [c_customer_sk#26] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#33] -Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#33 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34] - -(40) Filter [codegen id : 7] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#35, [id=#36] as decimal(32,6)))), DecimalType(38,8), true))) - -(41) Project [codegen id : 7] -Output [1]: [c_customer_sk#26] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34] - -(42) BroadcastExchange -Input [1]: [c_customer_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#37] - -(43) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#26] -Join condition: None - -(44) Scan parquet default.customer -Output [3]: [c_customer_sk#26, c_first_name#38, c_last_name#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(45) ColumnarToRow [codegen id : 11] -Input [3]: [c_customer_sk#26, c_first_name#38, c_last_name#39] - -(46) Filter [codegen id : 11] -Input [3]: [c_customer_sk#26, c_first_name#38, c_last_name#39] -Condition : isnotnull(c_customer_sk#26) - -(47) ReusedExchange [Reuses operator id: 38] -Output [3]: [c_customer_sk#26, sum#30, isEmpty#31] - -(48) HashAggregate [codegen id : 10] -Input [3]: [c_customer_sk#26, sum#30, isEmpty#31] -Keys [1]: [c_customer_sk#26] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#33] -Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#33 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34] - -(49) Filter [codegen id : 10] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#35, [id=#36] as decimal(32,6)))), DecimalType(38,8), true))) - -(50) Project [codegen id : 10] -Output [1]: [c_customer_sk#26 AS c_customer_sk#26#40] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#34] - -(51) BroadcastExchange -Input [1]: [c_customer_sk#26#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] - -(52) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_customer_sk#26] -Right keys [1]: [c_customer_sk#26#40] -Join condition: None - -(53) BroadcastExchange -Input [3]: [c_customer_sk#26, c_first_name#38, c_last_name#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#42] - -(54) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#26] -Join condition: None - -(55) Project [codegen id : 13] -Output [5]: [cs_sold_date_sk#1, cs_quantity#4, cs_list_price#5, c_first_name#38, c_last_name#39] -Input [7]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5, c_customer_sk#26, c_first_name#38, c_last_name#39] - -(56) Scan parquet default.date_dim -Output [3]: [d_date_sk#8, d_year#10, d_moy#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(57) ColumnarToRow [codegen id : 12] -Input [3]: [d_date_sk#8, d_year#10, d_moy#43] - -(58) Filter [codegen id : 12] -Input [3]: [d_date_sk#8, d_year#10, d_moy#43] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#43)) AND (d_year#10 = 2000)) AND (d_moy#43 = 2)) AND isnotnull(d_date_sk#8)) - -(59) Project [codegen id : 12] -Output [1]: [d_date_sk#8] -Input [3]: [d_date_sk#8, d_year#10, d_moy#43] - -(60) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] - -(61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(62) Project [codegen id : 13] -Output [4]: [cs_quantity#4, cs_list_price#5, c_first_name#38, c_last_name#39] -Input [6]: [cs_sold_date_sk#1, cs_quantity#4, cs_list_price#5, c_first_name#38, c_last_name#39, d_date_sk#8] - -(63) HashAggregate [codegen id : 13] -Input [4]: [cs_quantity#4, cs_list_price#5, c_first_name#38, c_last_name#39] -Keys [2]: [c_last_name#39, c_first_name#38] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [4]: [c_last_name#39, c_first_name#38, sum#47, isEmpty#48] - -(64) Exchange -Input [4]: [c_last_name#39, c_first_name#38, sum#47, isEmpty#48] -Arguments: hashpartitioning(c_last_name#39, c_first_name#38, 5), true, [id=#49] - -(65) HashAggregate [codegen id : 14] -Input [4]: [c_last_name#39, c_first_name#38, sum#47, isEmpty#48] -Keys [2]: [c_last_name#39, c_first_name#38] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))#50] -Results [3]: [c_last_name#39, c_first_name#38, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))#50 AS sales#51] - -(66) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#52, ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(67) ColumnarToRow [codegen id : 27] -Input [5]: [ws_sold_date_sk#52, ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] - -(68) Filter [codegen id : 27] -Input [5]: [ws_sold_date_sk#52, ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] -Condition : (isnotnull(ws_bill_customer_sk#54) AND isnotnull(ws_sold_date_sk#52)) - -(69) ReusedExchange [Reuses operator id: 25] -Output [1]: [item_sk#20] - -(70) BroadcastHashJoin [codegen id : 27] -Left keys [1]: [ws_item_sk#53] -Right keys [1]: [item_sk#20] -Join condition: None - -(71) Project [codegen id : 27] -Output [4]: [ws_sold_date_sk#52, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] -Input [5]: [ws_sold_date_sk#52, ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] - -(72) ReusedExchange [Reuses operator id: 38] -Output [3]: [c_customer_sk#26, sum#57, isEmpty#58] - -(73) HashAggregate [codegen id : 21] -Input [3]: [c_customer_sk#26, sum#57, isEmpty#58] -Keys [1]: [c_customer_sk#26] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#59] -Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#60] - -(74) Filter [codegen id : 21] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#35, [id=#36] as decimal(32,6)))), DecimalType(38,8), true))) - -(75) Project [codegen id : 21] -Output [1]: [c_customer_sk#26] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#60] - -(76) BroadcastExchange -Input [1]: [c_customer_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] - -(77) BroadcastHashJoin [codegen id : 27] -Left keys [1]: [ws_bill_customer_sk#54] -Right keys [1]: [c_customer_sk#26] -Join condition: None - -(78) Scan parquet default.customer -Output [3]: [c_customer_sk#26, c_first_name#38, c_last_name#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(79) ColumnarToRow [codegen id : 25] -Input [3]: [c_customer_sk#26, c_first_name#38, c_last_name#39] - -(80) Filter [codegen id : 25] -Input [3]: [c_customer_sk#26, c_first_name#38, c_last_name#39] -Condition : isnotnull(c_customer_sk#26) - -(81) ReusedExchange [Reuses operator id: 38] -Output [3]: [c_customer_sk#26, sum#57, isEmpty#58] - -(82) HashAggregate [codegen id : 24] -Input [3]: [c_customer_sk#26, sum#57, isEmpty#58] -Keys [1]: [c_customer_sk#26] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#59] -Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#60] - -(83) Filter [codegen id : 24] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#35, [id=#36] as decimal(32,6)))), DecimalType(38,8), true))) - -(84) Project [codegen id : 24] -Output [1]: [c_customer_sk#26 AS c_customer_sk#26#62] -Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#60] - -(85) BroadcastExchange -Input [1]: [c_customer_sk#26#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] - -(86) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [c_customer_sk#26] -Right keys [1]: [c_customer_sk#26#62] -Join condition: None - -(87) BroadcastExchange -Input [3]: [c_customer_sk#26, c_first_name#38, c_last_name#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] - -(88) BroadcastHashJoin [codegen id : 27] -Left keys [1]: [ws_bill_customer_sk#54] -Right keys [1]: [c_customer_sk#26] -Join condition: None - -(89) Project [codegen id : 27] -Output [5]: [ws_sold_date_sk#52, ws_quantity#55, ws_list_price#56, c_first_name#38, c_last_name#39] -Input [7]: [ws_sold_date_sk#52, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, c_customer_sk#26, c_first_name#38, c_last_name#39] - -(90) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#8] - -(91) BroadcastHashJoin [codegen id : 27] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(92) Project [codegen id : 27] -Output [4]: [ws_quantity#55, ws_list_price#56, c_first_name#38, c_last_name#39] -Input [6]: [ws_sold_date_sk#52, ws_quantity#55, ws_list_price#56, c_first_name#38, c_last_name#39, d_date_sk#8] - -(93) HashAggregate [codegen id : 27] -Input [4]: [ws_quantity#55, ws_list_price#56, c_first_name#38, c_last_name#39] -Keys [2]: [c_last_name#39, c_first_name#38] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [4]: [c_last_name#39, c_first_name#38, sum#67, isEmpty#68] - -(94) Exchange -Input [4]: [c_last_name#39, c_first_name#38, sum#67, isEmpty#68] -Arguments: hashpartitioning(c_last_name#39, c_first_name#38, 5), true, [id=#69] - -(95) HashAggregate [codegen id : 28] -Input [4]: [c_last_name#39, c_first_name#38, sum#67, isEmpty#68] -Keys [2]: [c_last_name#39, c_first_name#38] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#70] -Results [3]: [c_last_name#39, c_first_name#38, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#71] - -(96) Union - -(97) TakeOrderedAndProject -Input [3]: [c_last_name#39, c_first_name#38, sales#51] -Arguments: 100, [c_last_name#39 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, sales#51 ASC NULLS FIRST], [c_last_name#39, c_first_name#38, sales#51] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* HashAggregate (119) -+- Exchange (118) - +- * HashAggregate (117) - +- * HashAggregate (116) - +- Exchange (115) - +- * HashAggregate (114) - +- * Project (113) - +- * BroadcastHashJoin Inner BuildRight (112) - :- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Filter (100) - : : +- * ColumnarToRow (99) - : : +- Scan parquet default.store_sales (98) - : +- BroadcastExchange (104) - : +- * Filter (103) - : +- * ColumnarToRow (102) - : +- Scan parquet default.customer (101) - +- BroadcastExchange (111) - +- * Project (110) - +- * Filter (109) - +- * ColumnarToRow (108) - +- Scan parquet default.date_dim (107) - - -(98) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#6, ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(99) ColumnarToRow [codegen id : 3] -Input [4]: [ss_sold_date_sk#6, ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] - -(100) Filter [codegen id : 3] -Input [4]: [ss_sold_date_sk#6, ss_customer_sk#23, ss_quantity#24, ss_sales_price#25] -Condition : (isnotnull(ss_customer_sk#23) AND isnotnull(ss_sold_date_sk#6)) - -(101) Scan parquet default.customer -Output [1]: [c_customer_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(102) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#26] - -(103) Filter [codegen id : 1] -Input [1]: [c_customer_sk#26] -Condition : isnotnull(c_customer_sk#26) - -(104) BroadcastExchange -Input [1]: [c_customer_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] - -(105) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#23] -Right keys [1]: [c_customer_sk#26] -Join condition: None - -(106) Project [codegen id : 3] -Output [4]: [ss_sold_date_sk#6, ss_quantity#24, ss_sales_price#25, c_customer_sk#26] -Input [5]: [ss_sold_date_sk#6, ss_customer_sk#23, ss_quantity#24, ss_sales_price#25, c_customer_sk#26] - -(107) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_year#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(108) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#8, d_year#10] - -(109) Filter [codegen id : 2] -Input [2]: [d_date_sk#8, d_year#10] -Condition : (d_year#10 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#8)) - -(110) Project [codegen id : 2] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_year#10] - -(111) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] - -(112) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(113) Project [codegen id : 3] -Output [3]: [ss_quantity#24, ss_sales_price#25, c_customer_sk#26] -Input [5]: [ss_sold_date_sk#6, ss_quantity#24, ss_sales_price#25, c_customer_sk#26, d_date_sk#8] - -(114) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#24, ss_sales_price#25, c_customer_sk#26] -Keys [1]: [c_customer_sk#26] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#74, isEmpty#75] -Results [3]: [c_customer_sk#26, sum#76, isEmpty#77] - -(115) Exchange -Input [3]: [c_customer_sk#26, sum#76, isEmpty#77] -Arguments: hashpartitioning(c_customer_sk#26, 5), true, [id=#78] - -(116) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#26, sum#76, isEmpty#77] -Keys [1]: [c_customer_sk#26] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#79] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#24 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#25 as decimal(12,2)))), DecimalType(18,2), true))#79 AS csales#80] - -(117) HashAggregate [codegen id : 4] -Input [1]: [csales#80] -Keys: [] -Functions [1]: [partial_max(csales#80)] -Aggregate Attributes [1]: [max#81] -Results [1]: [max#82] - -(118) Exchange -Input [1]: [max#82] -Arguments: SinglePartition, true, [id=#83] - -(119) HashAggregate [codegen id : 5] -Input [1]: [max#82] -Keys: [] -Functions [1]: [max(csales#80)] -Aggregate Attributes [1]: [max(csales#80)#84] -Results [1]: [max(csales#80)#84 AS tpcds_cmax#85] - -Subquery:2 Hosting operator id = 49 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] - -Subquery:3 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] - -Subquery:4 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23b/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23b/simplified.txt deleted file mode 100644 index f879f38d5..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q23b/simplified.txt +++ /dev/null @@ -1,182 +0,0 @@ -TakeOrderedAndProject [c_last_name,c_first_name,sales] - Union - WholeStageCodegen (14) - HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (13) - HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_list_price,c_first_name,c_last_name] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_quantity,cs_list_price,c_first_name,c_last_name] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_quantity,cs_list_price] - BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (4) - Project [item_sk] - Filter [count(1)] - HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] - InputAdapter - Exchange [substr(i_item_desc, 1, 30),i_item_sk,d_date] #3 - WholeStageCodegen (3) - HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - Subquery #1 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - Exchange #9 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),csales,sum,isEmpty] - InputAdapter - Exchange [c_customer_sk] #10 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - InputAdapter - Exchange [c_customer_sk] #7 - WholeStageCodegen (6) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (11) - BroadcastHashJoin [c_customer_sk,c_customer_sk] - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (10) - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #7 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (12) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (28) - HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name] #16 - WholeStageCodegen (27) - HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_list_price,c_first_name,c_last_name] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_quantity,ws_list_price,c_first_name,c_last_name] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_sold_date_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] - BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] - InputAdapter - ReusedExchange [item_sk] #2 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (21) - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #7 - InputAdapter - BroadcastExchange #18 - WholeStageCodegen (25) - BroadcastHashJoin [c_customer_sk,c_customer_sk] - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (24) - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #7 - InputAdapter - ReusedExchange [d_date_sk] #15 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24a/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24a/explain.txt deleted file mode 100644 index ac5580565..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24a/explain.txt +++ /dev/null @@ -1,477 +0,0 @@ -== Physical Plan == -* Project (42) -+- * Filter (41) - +- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (28) - : +- * BroadcastHashJoin Inner BuildRight (27) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_returns (4) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.store (10) - : : +- BroadcastExchange (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.item (17) - : +- BroadcastExchange (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.customer (23) - +- BroadcastExchange (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.customer_address (29) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(3) Filter [codegen id : 6] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(4) Scan parquet default.store_returns -Output [2]: [sr_item_sk#6, sr_ticket_number#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [sr_item_sk#6, sr_ticket_number#7] - -(6) Filter [codegen id : 1] -Input [2]: [sr_item_sk#6, sr_ticket_number#7] -Condition : (isnotnull(sr_ticket_number#7) AND isnotnull(sr_item_sk#6)) - -(7) BroadcastExchange -Input [2]: [sr_item_sk#6, sr_ticket_number#7] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#8] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#7, sr_item_sk#6] -Join condition: None - -(9) Project [codegen id : 6] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#6, sr_ticket_number#7] - -(10) Scan parquet default.store -Output [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] - -(12) Filter [codegen id : 2] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] -Condition : (((isnotnull(s_market_id#11) AND (s_market_id#11 = 8)) AND isnotnull(s_store_sk#9)) AND isnotnull(s_zip#13)) - -(13) Project [codegen id : 2] -Output [4]: [s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] - -(14) BroadcastExchange -Input [4]: [s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] -Join condition: None - -(16) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13] -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] - -(17) Scan parquet default.item -Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] - -(19) Filter [codegen id : 3] -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale)) AND isnotnull(i_item_sk#15)) - -(20) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join condition: None - -(22) Project [codegen id : 6] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] - -(23) Scan parquet default.customer -Output [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(25) Filter [codegen id : 4] -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_birth_country#25)) - -(26) BroadcastExchange -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#22] -Join condition: None - -(28) Project [codegen id : 6] -Output [12]: [ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, c_birth_country#25] -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(29) Scan parquet default.customer_address -Output [3]: [ca_state#27, ca_zip#28, ca_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 5] -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] - -(31) Filter [codegen id : 5] -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] -Condition : (isnotnull(ca_country#29) AND isnotnull(ca_zip#28)) - -(32) BroadcastExchange -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#30] - -(33) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [c_birth_country#25, s_zip#13] -Right keys [2]: [upper(ca_country#29), ca_zip#28] -Join condition: None - -(34) Project [codegen id : 6] -Output [11]: [ss_net_paid#5, s_store_name#10, s_state#12, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] -Input [15]: [ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, c_birth_country#25, ca_state#27, ca_zip#28, ca_country#29] - -(35) HashAggregate [codegen id : 6] -Input [11]: [ss_net_paid#5, s_store_name#10, s_state#12, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] -Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#31] -Results [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#32] - -(36) Exchange -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#32] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), true, [id=#33] - -(37) HashAggregate [codegen id : 7] -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#32] -Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#34] -Results [4]: [c_last_name#24, c_first_name#23, s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#34,17,2) AS netpaid#35] - -(38) HashAggregate [codegen id : 7] -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#10, netpaid#35] -Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#10] -Functions [1]: [partial_sum(netpaid#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [5]: [c_last_name#24, c_first_name#23, s_store_name#10, sum#38, isEmpty#39] - -(39) Exchange -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#10, sum#38, isEmpty#39] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#10, 5), true, [id=#40] - -(40) HashAggregate [codegen id : 8] -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#10, sum#38, isEmpty#39] -Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#10] -Functions [1]: [sum(netpaid#35)] -Aggregate Attributes [1]: [sum(netpaid#35)#41] -Results [5]: [c_last_name#24, c_first_name#23, s_store_name#10, sum(netpaid#35)#41 AS paid#42, sum(netpaid#35)#41 AS sum(netpaid#35)#43] - -(41) Filter [codegen id : 8] -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#10, paid#42, sum(netpaid#35)#43] -Condition : (isnotnull(sum(netpaid#35)#43) AND (cast(sum(netpaid#35)#43 as decimal(33,8)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(33,8)))) - -(42) Project [codegen id : 8] -Output [4]: [c_last_name#24, c_first_name#23, s_store_name#10, paid#42] -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#10, paid#42, sum(netpaid#35)#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* HashAggregate (82) -+- Exchange (81) - +- * HashAggregate (80) - +- * HashAggregate (79) - +- Exchange (78) - +- * HashAggregate (77) - +- * Project (76) - +- * BroadcastHashJoin Inner BuildRight (75) - :- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (58) - : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : :- * Project (51) - : : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : : :- * Filter (45) - : : : : : +- * ColumnarToRow (44) - : : : : : +- Scan parquet default.store_sales (43) - : : : : +- BroadcastExchange (49) - : : : : +- * Filter (48) - : : : : +- * ColumnarToRow (47) - : : : : +- Scan parquet default.store_returns (46) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * Filter (54) - : : : +- * ColumnarToRow (53) - : : : +- Scan parquet default.store (52) - : : +- BroadcastExchange (62) - : : +- * Filter (61) - : : +- * ColumnarToRow (60) - : : +- Scan parquet default.item (59) - : +- BroadcastExchange (68) - : +- * Filter (67) - : +- * ColumnarToRow (66) - : +- Scan parquet default.customer (65) - +- BroadcastExchange (74) - +- * Filter (73) - +- * ColumnarToRow (72) - +- Scan parquet default.customer_address (71) - - -(43) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 6] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(45) Filter [codegen id : 6] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(46) Scan parquet default.store_returns -Output [2]: [sr_item_sk#6, sr_ticket_number#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 1] -Input [2]: [sr_item_sk#6, sr_ticket_number#7] - -(48) Filter [codegen id : 1] -Input [2]: [sr_item_sk#6, sr_ticket_number#7] -Condition : (isnotnull(sr_ticket_number#7) AND isnotnull(sr_item_sk#6)) - -(49) BroadcastExchange -Input [2]: [sr_item_sk#6, sr_ticket_number#7] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#46] - -(50) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#7, sr_item_sk#6] -Join condition: None - -(51) Project [codegen id : 6] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#6, sr_ticket_number#7] - -(52) Scan parquet default.store -Output [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] -ReadSchema: struct - -(53) ColumnarToRow [codegen id : 2] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] - -(54) Filter [codegen id : 2] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] -Condition : (((isnotnull(s_market_id#11) AND (s_market_id#11 = 8)) AND isnotnull(s_store_sk#9)) AND isnotnull(s_zip#13)) - -(55) Project [codegen id : 2] -Output [4]: [s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] - -(56) BroadcastExchange -Input [4]: [s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] - -(57) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] -Join condition: None - -(58) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13] -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] - -(59) Scan parquet default.item -Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(60) ColumnarToRow [codegen id : 3] -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] - -(61) Filter [codegen id : 3] -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Condition : isnotnull(i_item_sk#15) - -(62) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] - -(63) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join condition: None - -(64) Project [codegen id : 6] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] - -(65) Scan parquet default.customer -Output [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(66) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(67) Filter [codegen id : 4] -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_birth_country#25)) - -(68) BroadcastExchange -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] - -(69) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#22] -Join condition: None - -(70) Project [codegen id : 6] -Output [12]: [ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, c_birth_country#25] -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(71) Scan parquet default.customer_address -Output [3]: [ca_state#27, ca_zip#28, ca_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] -ReadSchema: struct - -(72) ColumnarToRow [codegen id : 5] -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] - -(73) Filter [codegen id : 5] -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] -Condition : (isnotnull(ca_country#29) AND isnotnull(ca_zip#28)) - -(74) BroadcastExchange -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#50] - -(75) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [c_birth_country#25, s_zip#13] -Right keys [2]: [upper(ca_country#29), ca_zip#28] -Join condition: None - -(76) Project [codegen id : 6] -Output [11]: [ss_net_paid#5, s_store_name#10, s_state#12, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] -Input [15]: [ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, c_birth_country#25, ca_state#27, ca_zip#28, ca_country#29] - -(77) HashAggregate [codegen id : 6] -Input [11]: [ss_net_paid#5, s_store_name#10, s_state#12, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] -Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#51] -Results [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#52] - -(78) Exchange -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#52] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), true, [id=#53] - -(79) HashAggregate [codegen id : 7] -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#52] -Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#54] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#54,17,2) AS netpaid#35] - -(80) HashAggregate [codegen id : 7] -Input [1]: [netpaid#35] -Keys: [] -Functions [1]: [partial_avg(netpaid#35)] -Aggregate Attributes [2]: [sum#55, count#56] -Results [2]: [sum#57, count#58] - -(81) Exchange -Input [2]: [sum#57, count#58] -Arguments: SinglePartition, true, [id=#59] - -(82) HashAggregate [codegen id : 8] -Input [2]: [sum#57, count#58] -Keys: [] -Functions [1]: [avg(netpaid#35)] -Aggregate Attributes [1]: [avg(netpaid#35)#60] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#35)#60)), DecimalType(24,8), true) AS (CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6)))#61] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24a/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24a/simplified.txt deleted file mode 100644 index 58b57dfaa..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24a/simplified.txt +++ /dev/null @@ -1,125 +0,0 @@ -WholeStageCodegen (8) - Project [c_last_name,c_first_name,s_store_name,paid] - Filter [sum(netpaid)] - Subquery #1 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg(netpaid),(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),sum,count] - InputAdapter - Exchange #8 - WholeStageCodegen (7) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #9 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) - Project [s_store_sk,s_store_name,s_state,s_zip] - Filter [s_market_id,s_store_sk,s_zip] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (4) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (5) - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum(netpaid),sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (7) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [s_store_sk,s_store_name,s_state,s_zip] - Filter [s_market_id,s_store_sk,s_zip] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [i_color,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24b/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24b/explain.txt deleted file mode 100644 index 0cf8c16a0..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24b/explain.txt +++ /dev/null @@ -1,477 +0,0 @@ -== Physical Plan == -* Project (42) -+- * Filter (41) - +- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (28) - : +- * BroadcastHashJoin Inner BuildRight (27) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_returns (4) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.store (10) - : : +- BroadcastExchange (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.item (17) - : +- BroadcastExchange (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.customer (23) - +- BroadcastExchange (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.customer_address (29) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(3) Filter [codegen id : 6] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(4) Scan parquet default.store_returns -Output [2]: [sr_item_sk#6, sr_ticket_number#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [sr_item_sk#6, sr_ticket_number#7] - -(6) Filter [codegen id : 1] -Input [2]: [sr_item_sk#6, sr_ticket_number#7] -Condition : (isnotnull(sr_ticket_number#7) AND isnotnull(sr_item_sk#6)) - -(7) BroadcastExchange -Input [2]: [sr_item_sk#6, sr_ticket_number#7] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#8] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#7, sr_item_sk#6] -Join condition: None - -(9) Project [codegen id : 6] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#6, sr_ticket_number#7] - -(10) Scan parquet default.store -Output [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] - -(12) Filter [codegen id : 2] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] -Condition : (((isnotnull(s_market_id#11) AND (s_market_id#11 = 8)) AND isnotnull(s_store_sk#9)) AND isnotnull(s_zip#13)) - -(13) Project [codegen id : 2] -Output [4]: [s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] - -(14) BroadcastExchange -Input [4]: [s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] -Join condition: None - -(16) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13] -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] - -(17) Scan parquet default.item -Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] - -(19) Filter [codegen id : 3] -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Condition : ((isnotnull(i_color#18) AND (i_color#18 = chiffon)) AND isnotnull(i_item_sk#15)) - -(20) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join condition: None - -(22) Project [codegen id : 6] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] - -(23) Scan parquet default.customer -Output [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(25) Filter [codegen id : 4] -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_birth_country#25)) - -(26) BroadcastExchange -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#22] -Join condition: None - -(28) Project [codegen id : 6] -Output [12]: [ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, c_birth_country#25] -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(29) Scan parquet default.customer_address -Output [3]: [ca_state#27, ca_zip#28, ca_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 5] -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] - -(31) Filter [codegen id : 5] -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] -Condition : (isnotnull(ca_country#29) AND isnotnull(ca_zip#28)) - -(32) BroadcastExchange -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#30] - -(33) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [c_birth_country#25, s_zip#13] -Right keys [2]: [upper(ca_country#29), ca_zip#28] -Join condition: None - -(34) Project [codegen id : 6] -Output [11]: [ss_net_paid#5, s_store_name#10, s_state#12, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] -Input [15]: [ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, c_birth_country#25, ca_state#27, ca_zip#28, ca_country#29] - -(35) HashAggregate [codegen id : 6] -Input [11]: [ss_net_paid#5, s_store_name#10, s_state#12, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] -Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#31] -Results [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#32] - -(36) Exchange -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#32] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), true, [id=#33] - -(37) HashAggregate [codegen id : 7] -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#32] -Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#34] -Results [4]: [c_last_name#24, c_first_name#23, s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#34,17,2) AS netpaid#35] - -(38) HashAggregate [codegen id : 7] -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#10, netpaid#35] -Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#10] -Functions [1]: [partial_sum(netpaid#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [5]: [c_last_name#24, c_first_name#23, s_store_name#10, sum#38, isEmpty#39] - -(39) Exchange -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#10, sum#38, isEmpty#39] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#10, 5), true, [id=#40] - -(40) HashAggregate [codegen id : 8] -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#10, sum#38, isEmpty#39] -Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#10] -Functions [1]: [sum(netpaid#35)] -Aggregate Attributes [1]: [sum(netpaid#35)#41] -Results [5]: [c_last_name#24, c_first_name#23, s_store_name#10, sum(netpaid#35)#41 AS paid#42, sum(netpaid#35)#41 AS sum(netpaid#35)#43] - -(41) Filter [codegen id : 8] -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#10, paid#42, sum(netpaid#35)#43] -Condition : (isnotnull(sum(netpaid#35)#43) AND (cast(sum(netpaid#35)#43 as decimal(33,8)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(33,8)))) - -(42) Project [codegen id : 8] -Output [4]: [c_last_name#24, c_first_name#23, s_store_name#10, paid#42] -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#10, paid#42, sum(netpaid#35)#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* HashAggregate (82) -+- Exchange (81) - +- * HashAggregate (80) - +- * HashAggregate (79) - +- Exchange (78) - +- * HashAggregate (77) - +- * Project (76) - +- * BroadcastHashJoin Inner BuildRight (75) - :- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (58) - : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : :- * Project (51) - : : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : : :- * Filter (45) - : : : : : +- * ColumnarToRow (44) - : : : : : +- Scan parquet default.store_sales (43) - : : : : +- BroadcastExchange (49) - : : : : +- * Filter (48) - : : : : +- * ColumnarToRow (47) - : : : : +- Scan parquet default.store_returns (46) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * Filter (54) - : : : +- * ColumnarToRow (53) - : : : +- Scan parquet default.store (52) - : : +- BroadcastExchange (62) - : : +- * Filter (61) - : : +- * ColumnarToRow (60) - : : +- Scan parquet default.item (59) - : +- BroadcastExchange (68) - : +- * Filter (67) - : +- * ColumnarToRow (66) - : +- Scan parquet default.customer (65) - +- BroadcastExchange (74) - +- * Filter (73) - +- * ColumnarToRow (72) - +- Scan parquet default.customer_address (71) - - -(43) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 6] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(45) Filter [codegen id : 6] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(46) Scan parquet default.store_returns -Output [2]: [sr_item_sk#6, sr_ticket_number#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 1] -Input [2]: [sr_item_sk#6, sr_ticket_number#7] - -(48) Filter [codegen id : 1] -Input [2]: [sr_item_sk#6, sr_ticket_number#7] -Condition : (isnotnull(sr_ticket_number#7) AND isnotnull(sr_item_sk#6)) - -(49) BroadcastExchange -Input [2]: [sr_item_sk#6, sr_ticket_number#7] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#46] - -(50) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#7, sr_item_sk#6] -Join condition: None - -(51) Project [codegen id : 6] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#6, sr_ticket_number#7] - -(52) Scan parquet default.store -Output [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] -ReadSchema: struct - -(53) ColumnarToRow [codegen id : 2] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] - -(54) Filter [codegen id : 2] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] -Condition : (((isnotnull(s_market_id#11) AND (s_market_id#11 = 8)) AND isnotnull(s_store_sk#9)) AND isnotnull(s_zip#13)) - -(55) Project [codegen id : 2] -Output [4]: [s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] -Input [5]: [s_store_sk#9, s_store_name#10, s_market_id#11, s_state#12, s_zip#13] - -(56) BroadcastExchange -Input [4]: [s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] - -(57) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] -Join condition: None - -(58) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13] -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#9, s_store_name#10, s_state#12, s_zip#13] - -(59) Scan parquet default.item -Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(60) ColumnarToRow [codegen id : 3] -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] - -(61) Filter [codegen id : 3] -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Condition : isnotnull(i_item_sk#15) - -(62) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] - -(63) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join condition: None - -(64) Project [codegen id : 6] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] - -(65) Scan parquet default.customer -Output [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(66) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(67) Filter [codegen id : 4] -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_birth_country#25)) - -(68) BroadcastExchange -Input [4]: [c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] - -(69) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#22] -Join condition: None - -(70) Project [codegen id : 6] -Output [12]: [ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, c_birth_country#25] -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(71) Scan parquet default.customer_address -Output [3]: [ca_state#27, ca_zip#28, ca_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] -ReadSchema: struct - -(72) ColumnarToRow [codegen id : 5] -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] - -(73) Filter [codegen id : 5] -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] -Condition : (isnotnull(ca_country#29) AND isnotnull(ca_zip#28)) - -(74) BroadcastExchange -Input [3]: [ca_state#27, ca_zip#28, ca_country#29] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#50] - -(75) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [c_birth_country#25, s_zip#13] -Right keys [2]: [upper(ca_country#29), ca_zip#28] -Join condition: None - -(76) Project [codegen id : 6] -Output [11]: [ss_net_paid#5, s_store_name#10, s_state#12, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] -Input [15]: [ss_net_paid#5, s_store_name#10, s_state#12, s_zip#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, c_birth_country#25, ca_state#27, ca_zip#28, ca_country#29] - -(77) HashAggregate [codegen id : 6] -Input [11]: [ss_net_paid#5, s_store_name#10, s_state#12, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] -Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#51] -Results [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#52] - -(78) Exchange -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#52] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), true, [id=#53] - -(79) HashAggregate [codegen id : 7] -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#52] -Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#10, ca_state#27, s_state#12, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#54] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#54,17,2) AS netpaid#35] - -(80) HashAggregate [codegen id : 7] -Input [1]: [netpaid#35] -Keys: [] -Functions [1]: [partial_avg(netpaid#35)] -Aggregate Attributes [2]: [sum#55, count#56] -Results [2]: [sum#57, count#58] - -(81) Exchange -Input [2]: [sum#57, count#58] -Arguments: SinglePartition, true, [id=#59] - -(82) HashAggregate [codegen id : 8] -Input [2]: [sum#57, count#58] -Keys: [] -Functions [1]: [avg(netpaid#35)] -Aggregate Attributes [1]: [avg(netpaid#35)#60] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#35)#60)), DecimalType(24,8), true) AS (CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6)))#61] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24b/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24b/simplified.txt deleted file mode 100644 index 58b57dfaa..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q24b/simplified.txt +++ /dev/null @@ -1,125 +0,0 @@ -WholeStageCodegen (8) - Project [c_last_name,c_first_name,s_store_name,paid] - Filter [sum(netpaid)] - Subquery #1 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg(netpaid),(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),sum,count] - InputAdapter - Exchange #8 - WholeStageCodegen (7) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #9 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) - Project [s_store_sk,s_store_name,s_state,s_zip] - Filter [s_market_id,s_store_sk,s_zip] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (4) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (5) - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum(netpaid),sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (7) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [s_store_sk,s_store_name,s_state,s_zip] - Filter [s_market_id,s_store_sk,s_zip] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [i_color,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q25/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q25/explain.txt deleted file mode 100644 index 6bdd709a7..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q25/explain.txt +++ /dev/null @@ -1,269 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (22) - : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet default.catalog_sales (10) - : : : : +- BroadcastExchange (20) - : : : : +- * Project (19) - : : : : +- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.date_dim (16) - : : : +- BroadcastExchange (27) - : : : +- * Project (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet default.date_dim (23) - : : +- ReusedExchange (30) - : +- BroadcastExchange (36) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.store (33) - +- BroadcastExchange (42) - +- * Filter (41) - +- * ColumnarToRow (40) - +- Scan parquet default.item (39) - - -(1) Scan parquet default.store_sales -Output [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_net_profit#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_net_profit#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_net_profit#6] -Condition : ((((isnotnull(ss_customer_sk#3) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#4)) - -(4) Scan parquet default.store_returns -Output [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11] - -(6) Filter [codegen id : 1] -Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11] -Condition : (((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) AND isnotnull(sr_returned_date_sk#7)) - -(7) BroadcastExchange -Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11] -Arguments: HashedRelationBroadcastMode(List(input[2, bigint, false], input[1, bigint, false], input[3, bigint, false]),false), [id=#12] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_net_profit#6, sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11] -Input [11]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_net_profit#6, sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11] - -(10) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16] - -(12) Filter [codegen id : 2] -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16] -Condition : ((isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) AND isnotnull(cs_sold_date_sk#13)) - -(13) BroadcastExchange -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint), cast(input[2, int, false] as bigint)),false), [id=#17] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cast(cs_bill_customer_sk#14 as bigint), cast(cs_item_sk#15 as bigint)] -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_net_profit#6, sr_returned_date_sk#7, sr_net_loss#11, cs_sold_date_sk#13, cs_net_profit#16] -Input [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_net_profit#6, sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16] - -(16) Scan parquet default.date_dim -Output [3]: [d_date_sk#18, d_year#19, d_moy#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#18, d_year#19, d_moy#20] - -(18) Filter [codegen id : 3] -Input [3]: [d_date_sk#18, d_year#19, d_moy#20] -Condition : ((((isnotnull(d_moy#20) AND isnotnull(d_year#19)) AND (d_moy#20 = 4)) AND (d_year#19 = 2001)) AND isnotnull(d_date_sk#18)) - -(19) Project [codegen id : 3] -Output [1]: [d_date_sk#18] -Input [3]: [d_date_sk#18, d_year#19, d_moy#20] - -(20) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] - -(21) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#18] -Join condition: None - -(22) Project [codegen id : 8] -Output [7]: [ss_item_sk#2, ss_store_sk#4, ss_net_profit#6, sr_returned_date_sk#7, sr_net_loss#11, cs_sold_date_sk#13, cs_net_profit#16] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_net_profit#6, sr_returned_date_sk#7, sr_net_loss#11, cs_sold_date_sk#13, cs_net_profit#16, d_date_sk#18] - -(23) Scan parquet default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] - -(25) Filter [codegen id : 4] -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(26) Project [codegen id : 4] -Output [1]: [d_date_sk#22] -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] - -(27) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] - -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#7] -Right keys [1]: [cast(d_date_sk#22 as bigint)] -Join condition: None - -(29) Project [codegen id : 8] -Output [6]: [ss_item_sk#2, ss_store_sk#4, ss_net_profit#6, sr_net_loss#11, cs_sold_date_sk#13, cs_net_profit#16] -Input [8]: [ss_item_sk#2, ss_store_sk#4, ss_net_profit#6, sr_returned_date_sk#7, sr_net_loss#11, cs_sold_date_sk#13, cs_net_profit#16, d_date_sk#22] - -(30) ReusedExchange [Reuses operator id: 27] -Output [1]: [d_date_sk#26] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#26] -Join condition: None - -(32) Project [codegen id : 8] -Output [5]: [ss_item_sk#2, ss_store_sk#4, ss_net_profit#6, sr_net_loss#11, cs_net_profit#16] -Input [7]: [ss_item_sk#2, ss_store_sk#4, ss_net_profit#6, sr_net_loss#11, cs_sold_date_sk#13, cs_net_profit#16, d_date_sk#26] - -(33) Scan parquet default.store -Output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] - -(35) Filter [codegen id : 6] -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Condition : isnotnull(s_store_sk#27) - -(36) BroadcastExchange -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#27] -Join condition: None - -(38) Project [codegen id : 8] -Output [6]: [ss_item_sk#2, ss_net_profit#6, sr_net_loss#11, cs_net_profit#16, s_store_id#28, s_store_name#29] -Input [8]: [ss_item_sk#2, ss_store_sk#4, ss_net_profit#6, sr_net_loss#11, cs_net_profit#16, s_store_sk#27, s_store_id#28, s_store_name#29] - -(39) Scan parquet default.item -Output [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(40) ColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] - -(41) Filter [codegen id : 7] -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Condition : isnotnull(i_item_sk#31) - -(42) BroadcastExchange -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] - -(43) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#31] -Join condition: None - -(44) Project [codegen id : 8] -Output [7]: [ss_net_profit#6, sr_net_loss#11, cs_net_profit#16, s_store_id#28, s_store_name#29, i_item_id#32, i_item_desc#33] -Input [9]: [ss_item_sk#2, ss_net_profit#6, sr_net_loss#11, cs_net_profit#16, s_store_id#28, s_store_name#29, i_item_sk#31, i_item_id#32, i_item_desc#33] - -(45) HashAggregate [codegen id : 8] -Input [7]: [ss_net_profit#6, sr_net_loss#11, cs_net_profit#16, s_store_id#28, s_store_name#29, i_item_id#32, i_item_desc#33] -Keys [4]: [i_item_id#32, i_item_desc#33, s_store_id#28, s_store_name#29] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum#35, sum#36, sum#37] -Results [7]: [i_item_id#32, i_item_desc#33, s_store_id#28, s_store_name#29, sum#38, sum#39, sum#40] - -(46) Exchange -Input [7]: [i_item_id#32, i_item_desc#33, s_store_id#28, s_store_name#29, sum#38, sum#39, sum#40] -Arguments: hashpartitioning(i_item_id#32, i_item_desc#33, s_store_id#28, s_store_name#29, 5), true, [id=#41] - -(47) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#32, i_item_desc#33, s_store_id#28, s_store_name#29, sum#38, sum#39, sum#40] -Keys [4]: [i_item_id#32, i_item_desc#33, s_store_id#28, s_store_name#29] -Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#42, sum(UnscaledValue(sr_net_loss#11))#43, sum(UnscaledValue(cs_net_profit#16))#44] -Results [7]: [i_item_id#32, i_item_desc#33, s_store_id#28, s_store_name#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#42,17,2) AS store_sales_profit#45, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#43,17,2) AS store_returns_loss#46, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#44,17,2) AS catalog_sales_profit#47] - -(48) TakeOrderedAndProject -Input [7]: [i_item_id#32, i_item_desc#33, s_store_id#28, s_store_name#29, store_sales_profit#45, store_returns_loss#46, catalog_sales_profit#47] -Arguments: 100, [i_item_id#32 ASC NULLS FIRST, i_item_desc#33 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#32, i_item_desc#33, s_store_id#28, s_store_name#29, store_sales_profit#45, store_returns_loss#46, catalog_sales_profit#47] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q25/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q25/simplified.txt deleted file mode 100644 index 4a40bdaff..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q25/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] - Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_sold_date_sk,cs_net_profit] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_returned_date_sk,sr_net_loss,cs_sold_date_sk,cs_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_net_profit,sr_returned_date_sk,sr_net_loss,cs_sold_date_sk,cs_net_profit] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_net_profit,sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q26/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q26/explain.txt deleted file mode 100644 index 12e953427..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q26/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * HashAggregate (33) - +- Exchange (32) - +- * HashAggregate (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.customer_demographics (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.date_dim (11) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.item (18) - +- BroadcastExchange (28) - +- * Project (27) - +- * Filter (26) - +- * ColumnarToRow (25) - +- Scan parquet default.promotion (24) - - -(1) Scan parquet default.catalog_sales -Output [8]: [cs_sold_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_promo_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [cs_sold_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_promo_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8] - -(3) Filter [codegen id : 5] -Input [8]: [cs_sold_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_promo_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8] -Condition : (((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#3)) AND isnotnull(cs_promo_sk#4)) - -(4) Scan parquet default.customer_demographics -Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -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)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] - -(6) Filter [codegen id : 1] -Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) - -(7) Project [codegen id : 1] -Output [1]: [cd_demo_sk#9] -Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#9] -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [cs_sold_date_sk#1, cs_item_sk#3, cs_promo_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8] -Input [9]: [cs_sold_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_promo_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cd_demo_sk#9] - -(11) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#14, d_year#15] - -(13) Filter [codegen id : 2] -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(14) Project [codegen id : 2] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] - -(15) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#14] -Join condition: None - -(17) Project [codegen id : 5] -Output [6]: [cs_item_sk#3, cs_promo_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8] -Input [8]: [cs_sold_date_sk#1, cs_item_sk#3, cs_promo_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, d_date_sk#14] - -(18) Scan parquet default.item -Output [2]: [i_item_sk#17, i_item_id#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#17, i_item_id#18] - -(20) Filter [codegen id : 3] -Input [2]: [i_item_sk#17, i_item_id#18] -Condition : isnotnull(i_item_sk#17) - -(21) BroadcastExchange -Input [2]: [i_item_sk#17, i_item_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] - -(22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#17] -Join condition: None - -(23) Project [codegen id : 5] -Output [6]: [cs_promo_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, i_item_id#18] -Input [8]: [cs_item_sk#3, cs_promo_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, i_item_sk#17, i_item_id#18] - -(24) Scan parquet default.promotion -Output [3]: [p_promo_sk#20, p_channel_email#21, p_channel_event#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [Or(EqualTo(p_channel_email,N),EqualTo(p_channel_event,N)), IsNotNull(p_promo_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 4] -Input [3]: [p_promo_sk#20, p_channel_email#21, p_channel_event#22] - -(26) Filter [codegen id : 4] -Input [3]: [p_promo_sk#20, p_channel_email#21, p_channel_event#22] -Condition : (((p_channel_email#21 = N) OR (p_channel_event#22 = N)) AND isnotnull(p_promo_sk#20)) - -(27) Project [codegen id : 4] -Output [1]: [p_promo_sk#20] -Input [3]: [p_promo_sk#20, p_channel_email#21, p_channel_event#22] - -(28) BroadcastExchange -Input [1]: [p_promo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] - -(29) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_promo_sk#4] -Right keys [1]: [p_promo_sk#20] -Join condition: None - -(30) Project [codegen id : 5] -Output [5]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, i_item_id#18] -Input [7]: [cs_promo_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, i_item_id#18, p_promo_sk#20] - -(31) HashAggregate [codegen id : 5] -Input [5]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(cast(cs_quantity#5 as bigint)), partial_avg(UnscaledValue(cs_list_price#6)), partial_avg(UnscaledValue(cs_coupon_amt#8)), partial_avg(UnscaledValue(cs_sales_price#7))] -Aggregate Attributes [8]: [sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] -Results [9]: [i_item_id#18, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] - -(32) Exchange -Input [9]: [i_item_id#18, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -Arguments: hashpartitioning(i_item_id#18, 5), true, [id=#40] - -(33) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#18, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(cast(cs_quantity#5 as bigint)), avg(UnscaledValue(cs_list_price#6)), avg(UnscaledValue(cs_coupon_amt#8)), avg(UnscaledValue(cs_sales_price#7))] -Aggregate Attributes [4]: [avg(cast(cs_quantity#5 as bigint))#41, avg(UnscaledValue(cs_list_price#6))#42, avg(UnscaledValue(cs_coupon_amt#8))#43, avg(UnscaledValue(cs_sales_price#7))#44] -Results [5]: [i_item_id#18, avg(cast(cs_quantity#5 as bigint))#41 AS agg1#45, cast((avg(UnscaledValue(cs_list_price#6))#42 / 100.0) as decimal(11,6)) AS agg2#46, cast((avg(UnscaledValue(cs_coupon_amt#8))#43 / 100.0) as decimal(11,6)) AS agg3#47, cast((avg(UnscaledValue(cs_sales_price#7))#44 / 100.0) as decimal(11,6)) AS agg4#48] - -(34) TakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#45, agg2#46, agg3#47, agg4#48] -Arguments: 100, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#45, agg2#46, agg3#47, agg4#48] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q26/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q26/simplified.txt deleted file mode 100644 index 94ce760c2..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q26/simplified.txt +++ /dev/null @@ -1,50 +0,0 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as bigint)),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_sold_date_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [cd_demo_sk] - Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [p_promo_sk] - Filter [p_channel_email,p_channel_event,p_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q27/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q27/explain.txt deleted file mode 100644 index 16aeff7a2..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q27/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * HashAggregate (33) - +- Exchange (32) - +- * HashAggregate (31) - +- * Expand (30) - +- * Project (29) - +- * BroadcastHashJoin Inner BuildRight (28) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.customer_demographics (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.date_dim (11) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.store (18) - +- BroadcastExchange (27) - +- * Filter (26) - +- * ColumnarToRow (25) - +- Scan parquet default.item (24) - - -(1) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -Condition : (((isnotnull(ss_cdemo_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_item_sk#2)) - -(4) Scan parquet default.customer_demographics -Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -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)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] - -(6) Filter [codegen id : 1] -Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) - -(7) Project [codegen id : 1] -Output [1]: [cd_demo_sk#9] -Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#9] -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, cd_demo_sk#9] - -(11) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#14, d_year#15] - -(13) Filter [codegen id : 2] -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(14) Project [codegen id : 2] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] - -(15) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#14] -Join condition: None - -(17) Project [codegen id : 5] -Output [6]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, d_date_sk#14] - -(18) Scan parquet default.store -Output [2]: [s_store_sk#17, s_state#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#17, s_state#18] - -(20) Filter [codegen id : 3] -Input [2]: [s_store_sk#17, s_state#18] -Condition : ((isnotnull(s_state#18) AND (s_state#18 = TN)) AND isnotnull(s_store_sk#17)) - -(21) BroadcastExchange -Input [2]: [s_store_sk#17, s_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] - -(22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#17] -Join condition: None - -(23) Project [codegen id : 5] -Output [6]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_state#18] -Input [8]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_store_sk#17, s_state#18] - -(24) Scan parquet default.item -Output [2]: [i_item_sk#20, i_item_id#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#20, i_item_id#21] - -(26) Filter [codegen id : 4] -Input [2]: [i_item_sk#20, i_item_id#21] -Condition : isnotnull(i_item_sk#20) - -(27) BroadcastExchange -Input [2]: [i_item_sk#20, i_item_id#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] - -(28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#20] -Join condition: None - -(29) Project [codegen id : 5] -Output [6]: [ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#21, s_state#18] -Input [8]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_state#18, i_item_sk#20, i_item_id#21] - -(30) Expand [codegen id : 5] -Input [6]: [ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#21, s_state#18] -Arguments: [List(ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#21, s_state#18, 0), List(ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#21, null, 1), List(ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, null, null, 3)], [ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#23, s_state#24, spark_grouping_id#25] - -(31) HashAggregate [codegen id : 5] -Input [7]: [ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#23, s_state#24, spark_grouping_id#25] -Keys [3]: [i_item_id#23, s_state#24, spark_grouping_id#25] -Functions [4]: [partial_avg(cast(ss_quantity#5 as bigint)), partial_avg(UnscaledValue(ss_list_price#6)), partial_avg(UnscaledValue(ss_coupon_amt#8)), partial_avg(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [8]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32, count#33] -Results [11]: [i_item_id#23, s_state#24, spark_grouping_id#25, sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40, count#41] - -(32) Exchange -Input [11]: [i_item_id#23, s_state#24, spark_grouping_id#25, sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40, count#41] -Arguments: hashpartitioning(i_item_id#23, s_state#24, spark_grouping_id#25, 5), true, [id=#42] - -(33) HashAggregate [codegen id : 6] -Input [11]: [i_item_id#23, s_state#24, spark_grouping_id#25, sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40, count#41] -Keys [3]: [i_item_id#23, s_state#24, spark_grouping_id#25] -Functions [4]: [avg(cast(ss_quantity#5 as bigint)), avg(UnscaledValue(ss_list_price#6)), avg(UnscaledValue(ss_coupon_amt#8)), avg(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [4]: [avg(cast(ss_quantity#5 as bigint))#43, avg(UnscaledValue(ss_list_price#6))#44, avg(UnscaledValue(ss_coupon_amt#8))#45, avg(UnscaledValue(ss_sales_price#7))#46] -Results [7]: [i_item_id#23, s_state#24, cast((shiftright(spark_grouping_id#25, 0) & 1) as tinyint) AS g_state#47, avg(cast(ss_quantity#5 as bigint))#43 AS agg1#48, cast((avg(UnscaledValue(ss_list_price#6))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(ss_coupon_amt#8))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(ss_sales_price#7))#46 / 100.0) as decimal(11,6)) AS agg4#51] - -(34) TakeOrderedAndProject -Input [7]: [i_item_id#23, s_state#24, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -Arguments: 100, [i_item_id#23 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST], [i_item_id#23, s_state#24, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q27/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q27/simplified.txt deleted file mode 100644 index 204094d44..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q27/simplified.txt +++ /dev/null @@ -1,50 +0,0 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id,s_state,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_sold_date_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [cd_demo_sk] - Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [s_state,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q28/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q28/explain.txt deleted file mode 100644 index 9788040bb..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q28/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -CollectLimit (71) -+- BroadcastNestedLoopJoin Inner BuildRight (70) - :- BroadcastNestedLoopJoin Inner BuildRight (58) - : :- BroadcastNestedLoopJoin Inner BuildRight (46) - : : :- BroadcastNestedLoopJoin Inner BuildRight (34) - : : : :- BroadcastNestedLoopJoin Inner BuildRight (22) - : : : : :- * HashAggregate (10) - : : : : : +- Exchange (9) - : : : : : +- * HashAggregate (8) - : : : : : +- * HashAggregate (7) - : : : : : +- Exchange (6) - : : : : : +- * HashAggregate (5) - : : : : : +- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (21) - : : : : +- * HashAggregate (20) - : : : : +- Exchange (19) - : : : : +- * HashAggregate (18) - : : : : +- * HashAggregate (17) - : : : : +- Exchange (16) - : : : : +- * HashAggregate (15) - : : : : +- * Project (14) - : : : : +- * Filter (13) - : : : : +- * ColumnarToRow (12) - : : : : +- Scan parquet default.store_sales (11) - : : : +- BroadcastExchange (33) - : : : +- * HashAggregate (32) - : : : +- Exchange (31) - : : : +- * HashAggregate (30) - : : : +- * HashAggregate (29) - : : : +- Exchange (28) - : : : +- * HashAggregate (27) - : : : +- * Project (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet default.store_sales (23) - : : +- BroadcastExchange (45) - : : +- * HashAggregate (44) - : : +- Exchange (43) - : : +- * HashAggregate (42) - : : +- * HashAggregate (41) - : : +- Exchange (40) - : : +- * HashAggregate (39) - : : +- * Project (38) - : : +- * Filter (37) - : : +- * ColumnarToRow (36) - : : +- Scan parquet default.store_sales (35) - : +- BroadcastExchange (57) - : +- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * HashAggregate (53) - : +- Exchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * Filter (49) - : +- * ColumnarToRow (48) - : +- Scan parquet default.store_sales (47) - +- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet default.store_sales (59) - - -(1) Scan parquet default.store_sales -Output [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(3) Filter [codegen id : 1] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (cast(ss_list_price#3 as decimal(12,2)) <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (cast(ss_coupon_amt#4 as decimal(12,2)) <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (cast(ss_wholesale_cost#2 as decimal(12,2)) <= 77.00)))) - -(4) Project [codegen id : 1] -Output [1]: [ss_list_price#3] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(5) HashAggregate [codegen id : 1] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#5, count(ss_list_price#3)#6] -Results [4]: [ss_list_price#3, sum#7, count#8, count#9] - -(6) Exchange -Input [4]: [ss_list_price#3, sum#7, count#8, count#9] -Arguments: hashpartitioning(ss_list_price#3, 5), true, [id=#10] - -(7) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#3, sum#7, count#8, count#9] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#5, count(ss_list_price#3)#6] -Results [4]: [ss_list_price#3, sum#7, count#8, count#9] - -(8) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#3, sum#7, count#8, count#9] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#5, count(ss_list_price#3)#6, count(ss_list_price#3)#11] -Results [4]: [sum#7, count#8, count#9, count#12] - -(9) Exchange -Input [4]: [sum#7, count#8, count#9, count#12] -Arguments: SinglePartition, true, [id=#13] - -(10) HashAggregate [codegen id : 3] -Input [4]: [sum#7, count#8, count#9, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#5, count(ss_list_price#3)#6, count(ss_list_price#3)#11] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#5 / 100.0) as decimal(11,6)) AS B1_LP#14, count(ss_list_price#3)#6 AS B1_CNT#15, count(ss_list_price#3)#11 AS B1_CNTD#16] - -(11) Scan parquet default.store_sales -Output [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(13) Filter [codegen id : 4] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (cast(ss_list_price#3 as decimal(12,2)) <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (cast(ss_coupon_amt#4 as decimal(12,2)) <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (cast(ss_wholesale_cost#2 as decimal(12,2)) <= 51.00)))) - -(14) Project [codegen id : 4] -Output [1]: [ss_list_price#3] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(15) HashAggregate [codegen id : 4] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#17, count(ss_list_price#3)#18] -Results [4]: [ss_list_price#3, sum#19, count#20, count#21] - -(16) Exchange -Input [4]: [ss_list_price#3, sum#19, count#20, count#21] -Arguments: hashpartitioning(ss_list_price#3, 5), true, [id=#22] - -(17) HashAggregate [codegen id : 5] -Input [4]: [ss_list_price#3, sum#19, count#20, count#21] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#17, count(ss_list_price#3)#18] -Results [4]: [ss_list_price#3, sum#19, count#20, count#21] - -(18) HashAggregate [codegen id : 5] -Input [4]: [ss_list_price#3, sum#19, count#20, count#21] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#17, count(ss_list_price#3)#18, count(ss_list_price#3)#23] -Results [4]: [sum#19, count#20, count#21, count#24] - -(19) Exchange -Input [4]: [sum#19, count#20, count#21, count#24] -Arguments: SinglePartition, true, [id=#25] - -(20) HashAggregate [codegen id : 6] -Input [4]: [sum#19, count#20, count#21, count#24] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#17, count(ss_list_price#3)#18, count(ss_list_price#3)#23] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#17 / 100.0) as decimal(11,6)) AS B2_LP#26, count(ss_list_price#3)#18 AS B2_CNT#27, count(ss_list_price#3)#23 AS B2_CNTD#28] - -(21) BroadcastExchange -Input [3]: [B2_LP#26, B2_CNT#27, B2_CNTD#28] -Arguments: IdentityBroadcastMode, [id=#29] - -(22) BroadcastNestedLoopJoin -Join condition: None - -(23) Scan parquet default.store_sales -Output [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 7] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(25) Filter [codegen id : 7] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (cast(ss_list_price#3 as decimal(12,2)) <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (cast(ss_coupon_amt#4 as decimal(12,2)) <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (cast(ss_wholesale_cost#2 as decimal(12,2)) <= 99.00)))) - -(26) Project [codegen id : 7] -Output [1]: [ss_list_price#3] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(27) HashAggregate [codegen id : 7] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#30, count(ss_list_price#3)#31] -Results [4]: [ss_list_price#3, sum#32, count#33, count#34] - -(28) Exchange -Input [4]: [ss_list_price#3, sum#32, count#33, count#34] -Arguments: hashpartitioning(ss_list_price#3, 5), true, [id=#35] - -(29) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#3, sum#32, count#33, count#34] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#30, count(ss_list_price#3)#31] -Results [4]: [ss_list_price#3, sum#32, count#33, count#34] - -(30) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#3, sum#32, count#33, count#34] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#30, count(ss_list_price#3)#31, count(ss_list_price#3)#36] -Results [4]: [sum#32, count#33, count#34, count#37] - -(31) Exchange -Input [4]: [sum#32, count#33, count#34, count#37] -Arguments: SinglePartition, true, [id=#38] - -(32) HashAggregate [codegen id : 9] -Input [4]: [sum#32, count#33, count#34, count#37] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#30, count(ss_list_price#3)#31, count(ss_list_price#3)#36] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#30 / 100.0) as decimal(11,6)) AS B3_LP#39, count(ss_list_price#3)#31 AS B3_CNT#40, count(ss_list_price#3)#36 AS B3_CNTD#41] - -(33) BroadcastExchange -Input [3]: [B3_LP#39, B3_CNT#40, B3_CNTD#41] -Arguments: IdentityBroadcastMode, [id=#42] - -(34) BroadcastNestedLoopJoin -Join condition: None - -(35) Scan parquet default.store_sales -Output [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(36) ColumnarToRow [codegen id : 10] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(37) Filter [codegen id : 10] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (cast(ss_list_price#3 as decimal(12,2)) <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (cast(ss_coupon_amt#4 as decimal(12,2)) <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (cast(ss_wholesale_cost#2 as decimal(12,2)) <= 58.00)))) - -(38) Project [codegen id : 10] -Output [1]: [ss_list_price#3] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(39) HashAggregate [codegen id : 10] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#43, count(ss_list_price#3)#44] -Results [4]: [ss_list_price#3, sum#45, count#46, count#47] - -(40) Exchange -Input [4]: [ss_list_price#3, sum#45, count#46, count#47] -Arguments: hashpartitioning(ss_list_price#3, 5), true, [id=#48] - -(41) HashAggregate [codegen id : 11] -Input [4]: [ss_list_price#3, sum#45, count#46, count#47] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#43, count(ss_list_price#3)#44] -Results [4]: [ss_list_price#3, sum#45, count#46, count#47] - -(42) HashAggregate [codegen id : 11] -Input [4]: [ss_list_price#3, sum#45, count#46, count#47] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#43, count(ss_list_price#3)#44, count(ss_list_price#3)#49] -Results [4]: [sum#45, count#46, count#47, count#50] - -(43) Exchange -Input [4]: [sum#45, count#46, count#47, count#50] -Arguments: SinglePartition, true, [id=#51] - -(44) HashAggregate [codegen id : 12] -Input [4]: [sum#45, count#46, count#47, count#50] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#43, count(ss_list_price#3)#44, count(ss_list_price#3)#49] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#43 / 100.0) as decimal(11,6)) AS B4_LP#52, count(ss_list_price#3)#44 AS B4_CNT#53, count(ss_list_price#3)#49 AS B4_CNTD#54] - -(45) BroadcastExchange -Input [3]: [B4_LP#52, B4_CNT#53, B4_CNTD#54] -Arguments: IdentityBroadcastMode, [id=#55] - -(46) BroadcastNestedLoopJoin -Join condition: None - -(47) Scan parquet default.store_sales -Output [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25)] -ReadSchema: struct - -(48) ColumnarToRow [codegen id : 13] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(49) Filter [codegen id : 13] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (cast(ss_list_price#3 as decimal(12,2)) <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (cast(ss_coupon_amt#4 as decimal(12,2)) <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (cast(ss_wholesale_cost#2 as decimal(12,2)) <= 37.00)))) - -(50) Project [codegen id : 13] -Output [1]: [ss_list_price#3] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(51) HashAggregate [codegen id : 13] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#56, count(ss_list_price#3)#57] -Results [4]: [ss_list_price#3, sum#58, count#59, count#60] - -(52) Exchange -Input [4]: [ss_list_price#3, sum#58, count#59, count#60] -Arguments: hashpartitioning(ss_list_price#3, 5), true, [id=#61] - -(53) HashAggregate [codegen id : 14] -Input [4]: [ss_list_price#3, sum#58, count#59, count#60] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#56, count(ss_list_price#3)#57] -Results [4]: [ss_list_price#3, sum#58, count#59, count#60] - -(54) HashAggregate [codegen id : 14] -Input [4]: [ss_list_price#3, sum#58, count#59, count#60] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#56, count(ss_list_price#3)#57, count(ss_list_price#3)#62] -Results [4]: [sum#58, count#59, count#60, count#63] - -(55) Exchange -Input [4]: [sum#58, count#59, count#60, count#63] -Arguments: SinglePartition, true, [id=#64] - -(56) HashAggregate [codegen id : 15] -Input [4]: [sum#58, count#59, count#60, count#63] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#56, count(ss_list_price#3)#57, count(ss_list_price#3)#62] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#56 / 100.0) as decimal(11,6)) AS B5_LP#65, count(ss_list_price#3)#57 AS B5_CNT#66, count(ss_list_price#3)#62 AS B5_CNTD#67] - -(57) BroadcastExchange -Input [3]: [B5_LP#65, B5_CNT#66, B5_CNTD#67] -Arguments: IdentityBroadcastMode, [id=#68] - -(58) BroadcastNestedLoopJoin -Join condition: None - -(59) Scan parquet default.store_sales -Output [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30)] -ReadSchema: struct - -(60) ColumnarToRow [codegen id : 16] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(61) Filter [codegen id : 16] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (cast(ss_list_price#3 as decimal(12,2)) <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (cast(ss_coupon_amt#4 as decimal(12,2)) <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (cast(ss_wholesale_cost#2 as decimal(12,2)) <= 27.00)))) - -(62) Project [codegen id : 16] -Output [1]: [ss_list_price#3] -Input [4]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4] - -(63) HashAggregate [codegen id : 16] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#69, count(ss_list_price#3)#70] -Results [4]: [ss_list_price#3, sum#71, count#72, count#73] - -(64) Exchange -Input [4]: [ss_list_price#3, sum#71, count#72, count#73] -Arguments: hashpartitioning(ss_list_price#3, 5), true, [id=#74] - -(65) HashAggregate [codegen id : 17] -Input [4]: [ss_list_price#3, sum#71, count#72, count#73] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#69, count(ss_list_price#3)#70] -Results [4]: [ss_list_price#3, sum#71, count#72, count#73] - -(66) HashAggregate [codegen id : 17] -Input [4]: [ss_list_price#3, sum#71, count#72, count#73] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#69, count(ss_list_price#3)#70, count(ss_list_price#3)#75] -Results [4]: [sum#71, count#72, count#73, count#76] - -(67) Exchange -Input [4]: [sum#71, count#72, count#73, count#76] -Arguments: SinglePartition, true, [id=#77] - -(68) HashAggregate [codegen id : 18] -Input [4]: [sum#71, count#72, count#73, count#76] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#69, count(ss_list_price#3)#70, count(ss_list_price#3)#75] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#69 / 100.0) as decimal(11,6)) AS B6_LP#78, count(ss_list_price#3)#70 AS B6_CNT#79, count(ss_list_price#3)#75 AS B6_CNTD#80] - -(69) BroadcastExchange -Input [3]: [B6_LP#78, B6_CNT#79, B6_CNTD#80] -Arguments: IdentityBroadcastMode, [id=#81] - -(70) BroadcastNestedLoopJoin -Join condition: None - -(71) CollectLimit -Input [18]: [B1_LP#14, B1_CNT#15, B1_CNTD#16, B2_LP#26, B2_CNT#27, B2_CNTD#28, B3_LP#39, B3_CNT#40, B3_CNTD#41, B4_LP#52, B4_CNT#53, B4_CNTD#54, B5_LP#65, B5_CNT#66, B5_CNTD#67, B6_LP#78, B6_CNT#79, B6_CNTD#80] -Arguments: 100 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q28/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q28/simplified.txt deleted file mode 100644 index d896002b0..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q28/simplified.txt +++ /dev/null @@ -1,107 +0,0 @@ -CollectLimit - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - WholeStageCodegen (3) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] - InputAdapter - Exchange #1 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #2 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - BroadcastExchange #3 - WholeStageCodegen (6) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] - InputAdapter - Exchange #4 - WholeStageCodegen (5) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #5 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - BroadcastExchange #6 - WholeStageCodegen (9) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] - InputAdapter - Exchange #7 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #8 - WholeStageCodegen (7) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - BroadcastExchange #9 - WholeStageCodegen (12) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] - InputAdapter - Exchange #10 - WholeStageCodegen (11) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #11 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - BroadcastExchange #12 - WholeStageCodegen (15) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] - InputAdapter - Exchange #13 - WholeStageCodegen (14) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #14 - WholeStageCodegen (13) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - BroadcastExchange #15 - WholeStageCodegen (18) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] - InputAdapter - Exchange #16 - WholeStageCodegen (17) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #17 - WholeStageCodegen (16) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - Project [ss_list_price] - Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q29/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q29/explain.txt deleted file mode 100644 index d1d99897d..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q29/explain.txt +++ /dev/null @@ -1,292 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (22) - : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet default.catalog_sales (10) - : : : : +- BroadcastExchange (20) - : : : : +- * Project (19) - : : : : +- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.date_dim (16) - : : : +- BroadcastExchange (27) - : : : +- * Project (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet default.date_dim (23) - : : +- BroadcastExchange (34) - : : +- * Project (33) - : : +- * Filter (32) - : : +- * ColumnarToRow (31) - : : +- Scan parquet default.date_dim (30) - : +- BroadcastExchange (40) - : +- * Filter (39) - : +- * ColumnarToRow (38) - : +- Scan parquet default.store (37) - +- BroadcastExchange (46) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet default.item (43) - - -(1) Scan parquet default.store_sales -Output [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6] -Condition : ((((isnotnull(ss_customer_sk#3) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#4)) - -(4) Scan parquet default.store_returns -Output [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] - -(6) Filter [codegen id : 1] -Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] -Condition : (((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) AND isnotnull(sr_returned_date_sk#7)) - -(7) BroadcastExchange -Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] -Arguments: HashedRelationBroadcastMode(List(input[2, bigint, false], input[1, bigint, false], input[3, bigint, false]),false), [id=#12] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11] -Input [11]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6, sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] - -(10) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16] - -(12) Filter [codegen id : 2] -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16] -Condition : ((isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) AND isnotnull(cs_sold_date_sk#13)) - -(13) BroadcastExchange -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint), cast(input[2, int, false] as bigint)),false), [id=#17] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cast(cs_bill_customer_sk#14 as bigint), cast(cs_item_sk#15 as bigint)] -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16] -Input [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, cs_sold_date_sk#13, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16] - -(16) Scan parquet default.date_dim -Output [3]: [d_date_sk#18, d_year#19, d_moy#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#18, d_year#19, d_moy#20] - -(18) Filter [codegen id : 3] -Input [3]: [d_date_sk#18, d_year#19, d_moy#20] -Condition : ((((isnotnull(d_moy#20) AND isnotnull(d_year#19)) AND (d_moy#20 = 9)) AND (d_year#19 = 1999)) AND isnotnull(d_date_sk#18)) - -(19) Project [codegen id : 3] -Output [1]: [d_date_sk#18] -Input [3]: [d_date_sk#18, d_year#19, d_moy#20] - -(20) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] - -(21) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#18] -Join condition: None - -(22) Project [codegen id : 8] -Output [7]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16, d_date_sk#18] - -(23) Scan parquet default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] - -(25) Filter [codegen id : 4] -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(26) Project [codegen id : 4] -Output [1]: [d_date_sk#22] -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] - -(27) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] - -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#7] -Right keys [1]: [cast(d_date_sk#22 as bigint)] -Join condition: None - -(29) Project [codegen id : 8] -Output [6]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16] -Input [8]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_returned_date_sk#7, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16, d_date_sk#22] - -(30) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#26, d_year#27] - -(32) Filter [codegen id : 5] -Input [2]: [d_date_sk#26, d_year#27] -Condition : (d_year#27 IN (1999,2000,2001) AND isnotnull(d_date_sk#26)) - -(33) Project [codegen id : 5] -Output [1]: [d_date_sk#26] -Input [2]: [d_date_sk#26, d_year#27] - -(34) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#26] -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_return_quantity#11, cs_sold_date_sk#13, cs_quantity#16, d_date_sk#26] - -(37) Scan parquet default.store -Output [3]: [s_store_sk#29, s_store_id#30, s_store_name#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#29, s_store_id#30, s_store_name#31] - -(39) Filter [codegen id : 6] -Input [3]: [s_store_sk#29, s_store_id#30, s_store_name#31] -Condition : isnotnull(s_store_sk#29) - -(40) BroadcastExchange -Input [3]: [s_store_sk#29, s_store_id#30, s_store_name#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] - -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#29] -Join condition: None - -(42) Project [codegen id : 8] -Output [6]: [ss_item_sk#2, ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#31] -Input [8]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_store_sk#29, s_store_id#30, s_store_name#31] - -(43) Scan parquet default.item -Output [3]: [i_item_sk#33, i_item_id#34, i_item_desc#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#33, i_item_id#34, i_item_desc#35] - -(45) Filter [codegen id : 7] -Input [3]: [i_item_sk#33, i_item_id#34, i_item_desc#35] -Condition : isnotnull(i_item_sk#33) - -(46) BroadcastExchange -Input [3]: [i_item_sk#33, i_item_id#34, i_item_desc#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] - -(47) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#33] -Join condition: None - -(48) Project [codegen id : 8] -Output [7]: [ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#31, i_item_id#34, i_item_desc#35] -Input [9]: [ss_item_sk#2, ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#31, i_item_sk#33, i_item_id#34, i_item_desc#35] - -(49) HashAggregate [codegen id : 8] -Input [7]: [ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#31, i_item_id#34, i_item_desc#35] -Keys [4]: [i_item_id#34, i_item_desc#35, s_store_id#30, s_store_name#31] -Functions [3]: [partial_sum(cast(ss_quantity#6 as bigint)), partial_sum(cast(sr_return_quantity#11 as bigint)), partial_sum(cast(cs_quantity#16 as bigint))] -Aggregate Attributes [3]: [sum#37, sum#38, sum#39] -Results [7]: [i_item_id#34, i_item_desc#35, s_store_id#30, s_store_name#31, sum#40, sum#41, sum#42] - -(50) Exchange -Input [7]: [i_item_id#34, i_item_desc#35, s_store_id#30, s_store_name#31, sum#40, sum#41, sum#42] -Arguments: hashpartitioning(i_item_id#34, i_item_desc#35, s_store_id#30, s_store_name#31, 5), true, [id=#43] - -(51) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#34, i_item_desc#35, s_store_id#30, s_store_name#31, sum#40, sum#41, sum#42] -Keys [4]: [i_item_id#34, i_item_desc#35, s_store_id#30, s_store_name#31] -Functions [3]: [sum(cast(ss_quantity#6 as bigint)), sum(cast(sr_return_quantity#11 as bigint)), sum(cast(cs_quantity#16 as bigint))] -Aggregate Attributes [3]: [sum(cast(ss_quantity#6 as bigint))#44, sum(cast(sr_return_quantity#11 as bigint))#45, sum(cast(cs_quantity#16 as bigint))#46] -Results [7]: [i_item_id#34, i_item_desc#35, s_store_id#30, s_store_name#31, sum(cast(ss_quantity#6 as bigint))#44 AS store_sales_quantity#47, sum(cast(sr_return_quantity#11 as bigint))#45 AS store_returns_quantity#48, sum(cast(cs_quantity#16 as bigint))#46 AS catalog_sales_quantity#49] - -(52) TakeOrderedAndProject -Input [7]: [i_item_id#34, i_item_desc#35, s_store_id#30, s_store_name#31, store_sales_quantity#47, store_returns_quantity#48, catalog_sales_quantity#49] -Arguments: 100, [i_item_id#34 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, s_store_name#31 ASC NULLS FIRST], [i_item_id#34, i_item_desc#35, s_store_id#30, s_store_name#31, store_sales_quantity#47, store_returns_quantity#48, catalog_sales_quantity#49] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q29/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q29/simplified.txt deleted file mode 100644 index 26e4e3a55..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q29/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(cast(ss_quantity as bigint)),sum(cast(sr_return_quantity as bigint)),sum(cast(cs_quantity as bigint)),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_sold_date_sk,cs_quantity] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_return_quantity,cs_sold_date_sk,cs_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_return_quantity,cs_sold_date_sk,cs_quantity] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q3/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q3/explain.txt deleted file mode 100644 index bec06d146..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q3/explain.txt +++ /dev/null @@ -1,122 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.date_dim (1) - : +- BroadcastExchange (8) - : +- * Filter (7) - : +- * ColumnarToRow (6) - : +- Scan parquet default.store_sales (5) - +- BroadcastExchange (15) - +- * Project (14) - +- * Filter (13) - +- * ColumnarToRow (12) - +- Scan parquet default.item (11) - - -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] - -(3) Filter [codegen id : 3] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) - -(4) Project [codegen id : 3] -Output [2]: [d_date_sk#1, d_year#2] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] - -(5) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] - -(7) Filter [codegen id : 1] -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Condition : (isnotnull(ss_sold_date_sk#4) AND isnotnull(ss_item_sk#5)) - -(8) BroadcastExchange -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#5, ss_ext_sales_price#6] -Input [5]: [d_date_sk#1, d_year#2, ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] - -(11) Scan parquet default.item -Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manufact_id#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manufact_id#11] - -(13) Filter [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manufact_id#11] -Condition : ((isnotnull(i_manufact_id#11) AND (i_manufact_id#11 = 128)) AND isnotnull(i_item_sk#8)) - -(14) Project [codegen id : 2] -Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manufact_id#11] - -(15) BroadcastExchange -Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#8] -Join condition: None - -(17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#6, i_brand_id#9, i_brand#10] -Input [6]: [d_year#2, ss_item_sk#5, ss_ext_sales_price#6, i_item_sk#8, i_brand_id#9, i_brand#10] - -(18) HashAggregate [codegen id : 3] -Input [4]: [d_year#2, ss_ext_sales_price#6, i_brand_id#9, i_brand#10] -Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum#13] -Results [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] - -(19) Exchange -Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] -Arguments: hashpartitioning(d_year#2, i_brand#10, i_brand_id#9, 5), true, [id=#15] - -(20) HashAggregate [codegen id : 4] -Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] -Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#16] -Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#16,17,2) AS sum_agg#19] - -(21) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#17, brand#18, sum_agg#19] -Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, sum_agg#19] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q3/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q3/simplified.txt deleted file mode 100644 index ed3a06904..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q3/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] - WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] - InputAdapter - Exchange [d_year,i_brand,i_brand_id] #1 - WholeStageCodegen (3) - HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [d_date_sk,d_year] - Filter [d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manufact_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q30/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q30/explain.txt deleted file mode 100644 index fffcc5ca3..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q30/explain.txt +++ /dev/null @@ -1,303 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (53) -+- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (20) - : : : +- * HashAggregate (19) - : : : +- Exchange (18) - : : : +- * HashAggregate (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.web_returns (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.customer_address (11) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- Exchange (34) - : : +- * HashAggregate (33) - : : +- * HashAggregate (32) - : : +- Exchange (31) - : : +- * HashAggregate (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (26) - : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : :- * Filter (23) - : : : : +- * ColumnarToRow (22) - : : : : +- Scan parquet default.web_returns (21) - : : : +- ReusedExchange (24) - : : +- ReusedExchange (27) - : +- BroadcastExchange (43) - : +- * Filter (42) - : +- * ColumnarToRow (41) - : +- Scan parquet default.customer (40) - +- BroadcastExchange (50) - +- * Project (49) - +- * Filter (48) - +- * ColumnarToRow (47) - +- Scan parquet default.customer_address (46) - - -(1) Scan parquet default.web_returns -Output [4]: [wr_returned_date_sk#1, wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_returned_date_sk), IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [wr_returned_date_sk#1, wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4] - -(3) Filter [codegen id : 3] -Input [4]: [wr_returned_date_sk#1, wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4] -Condition : ((isnotnull(wr_returned_date_sk#1) AND isnotnull(wr_returning_addr_sk#3)) AND isnotnull(wr_returning_customer_sk#2)) - -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_year#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_year#6] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_year#6] -Condition : ((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND isnotnull(d_date_sk#5)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_year#6] - -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returned_date_sk#1] -Right keys [1]: [cast(d_date_sk#5 as bigint)] -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4] -Input [5]: [wr_returned_date_sk#1, wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4, d_date_sk#5] - -(11) Scan parquet default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#8, ca_state#9] - -(13) Filter [codegen id : 2] -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_state#9)) - -(14) BroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returning_addr_sk#3] -Right keys [1]: [cast(ca_address_sk#8 as bigint)] -Join condition: None - -(16) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#2, wr_return_amt#4, ca_state#9] -Input [5]: [wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4, ca_address_sk#8, ca_state#9] - -(17) HashAggregate [codegen id : 3] -Input [3]: [wr_returning_customer_sk#2, wr_return_amt#4, ca_state#9] -Keys [2]: [wr_returning_customer_sk#2, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#4))] -Aggregate Attributes [1]: [sum#11] -Results [3]: [wr_returning_customer_sk#2, ca_state#9, sum#12] - -(18) Exchange -Input [3]: [wr_returning_customer_sk#2, ca_state#9, sum#12] -Arguments: hashpartitioning(wr_returning_customer_sk#2, ca_state#9, 5), true, [id=#13] - -(19) HashAggregate [codegen id : 11] -Input [3]: [wr_returning_customer_sk#2, ca_state#9, sum#12] -Keys [2]: [wr_returning_customer_sk#2, ca_state#9] -Functions [1]: [sum(UnscaledValue(wr_return_amt#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#4))#14] -Results [3]: [wr_returning_customer_sk#2 AS ctr_customer_sk#15, ca_state#9 AS ctr_state#16, MakeDecimal(sum(UnscaledValue(wr_return_amt#4))#14,17,2) AS ctr_total_return#17] - -(20) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#15, ctr_state#16, ctr_total_return#17] -Condition : isnotnull(ctr_total_return#17) - -(21) Scan parquet default.web_returns -Output [4]: [wr_returned_date_sk#1, wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_returned_date_sk), IsNotNull(wr_returning_addr_sk)] -ReadSchema: struct - -(22) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returned_date_sk#1, wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4] - -(23) Filter [codegen id : 6] -Input [4]: [wr_returned_date_sk#1, wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4] -Condition : (isnotnull(wr_returned_date_sk#1) AND isnotnull(wr_returning_addr_sk#3)) - -(24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#1] -Right keys [1]: [cast(d_date_sk#5 as bigint)] -Join condition: None - -(26) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4] -Input [5]: [wr_returned_date_sk#1, wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4, d_date_sk#5] - -(27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#8, ca_state#9] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#3] -Right keys [1]: [cast(ca_address_sk#8 as bigint)] -Join condition: None - -(29) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#2, wr_return_amt#4, ca_state#9] -Input [5]: [wr_returning_customer_sk#2, wr_returning_addr_sk#3, wr_return_amt#4, ca_address_sk#8, ca_state#9] - -(30) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#2, wr_return_amt#4, ca_state#9] -Keys [2]: [wr_returning_customer_sk#2, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#4))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [wr_returning_customer_sk#2, ca_state#9, sum#19] - -(31) Exchange -Input [3]: [wr_returning_customer_sk#2, ca_state#9, sum#19] -Arguments: hashpartitioning(wr_returning_customer_sk#2, ca_state#9, 5), true, [id=#20] - -(32) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#2, ca_state#9, sum#19] -Keys [2]: [wr_returning_customer_sk#2, ca_state#9] -Functions [1]: [sum(UnscaledValue(wr_return_amt#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#4))#21] -Results [2]: [ca_state#9 AS ctr_state#16, MakeDecimal(sum(UnscaledValue(wr_return_amt#4))#21,17,2) AS ctr_total_return#17] - -(33) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#16, ctr_total_return#17] -Keys [1]: [ctr_state#16] -Functions [1]: [partial_avg(ctr_total_return#17)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [3]: [ctr_state#16, sum#24, count#25] - -(34) Exchange -Input [3]: [ctr_state#16, sum#24, count#25] -Arguments: hashpartitioning(ctr_state#16, 5), true, [id=#26] - -(35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#16, sum#24, count#25] -Keys [1]: [ctr_state#16] -Functions [1]: [avg(ctr_total_return#17)] -Aggregate Attributes [1]: [avg(ctr_total_return#17)#27] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#17)#27) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28, ctr_state#16 AS ctr_state#16#29] - -(36) Filter [codegen id : 8] -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28, ctr_state#16#29] -Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28) - -(37) BroadcastExchange -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28, ctr_state#16#29] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#30] - -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#16] -Right keys [1]: [ctr_state#16#29] -Join condition: (cast(ctr_total_return#17 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28) - -(39) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#15, ctr_total_return#17] -Input [5]: [ctr_customer_sk#15, ctr_state#16, ctr_total_return#17, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28, ctr_state#16#29] - -(40) Scan parquet default.customer -Output [14]: [c_customer_sk#31, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_day#38, c_birth_month#39, c_birth_year#40, c_birth_country#41, c_login#42, c_email_address#43, c_last_review_date#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#31, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_day#38, c_birth_month#39, c_birth_year#40, c_birth_country#41, c_login#42, c_email_address#43, c_last_review_date#44] - -(42) Filter [codegen id : 9] -Input [14]: [c_customer_sk#31, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_day#38, c_birth_month#39, c_birth_year#40, c_birth_country#41, c_login#42, c_email_address#43, c_last_review_date#44] -Condition : (isnotnull(c_customer_sk#31) AND isnotnull(c_current_addr_sk#33)) - -(43) BroadcastExchange -Input [14]: [c_customer_sk#31, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_day#38, c_birth_month#39, c_birth_year#40, c_birth_country#41, c_login#42, c_email_address#43, c_last_review_date#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] - -(44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#15] -Right keys [1]: [cast(c_customer_sk#31 as bigint)] -Join condition: None - -(45) Project [codegen id : 11] -Output [14]: [ctr_total_return#17, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_day#38, c_birth_month#39, c_birth_year#40, c_birth_country#41, c_login#42, c_email_address#43, c_last_review_date#44] -Input [16]: [ctr_customer_sk#15, ctr_total_return#17, c_customer_sk#31, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_day#38, c_birth_month#39, c_birth_year#40, c_birth_country#41, c_login#42, c_email_address#43, c_last_review_date#44] - -(46) Scan parquet default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#8, ca_state#9] - -(48) Filter [codegen id : 10] -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : ((isnotnull(ca_state#9) AND (ca_state#9 = GA)) AND isnotnull(ca_address_sk#8)) - -(49) Project [codegen id : 10] -Output [1]: [ca_address_sk#8] -Input [2]: [ca_address_sk#8, ca_state#9] - -(50) BroadcastExchange -Input [1]: [ca_address_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] - -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#8] -Join condition: None - -(52) Project [codegen id : 11] -Output [13]: [c_customer_id#32, c_salutation#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_day#38, c_birth_month#39, c_birth_year#40, c_birth_country#41, c_login#42, c_email_address#43, c_last_review_date#44, ctr_total_return#17] -Input [15]: [ctr_total_return#17, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_day#38, c_birth_month#39, c_birth_year#40, c_birth_country#41, c_login#42, c_email_address#43, c_last_review_date#44, ca_address_sk#8] - -(53) TakeOrderedAndProject -Input [13]: [c_customer_id#32, c_salutation#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_day#38, c_birth_month#39, c_birth_year#40, c_birth_country#41, c_login#42, c_email_address#43, c_last_review_date#44, ctr_total_return#17] -Arguments: 100, [c_customer_id#32 ASC NULLS FIRST, c_salutation#34 ASC NULLS FIRST, c_first_name#35 ASC NULLS FIRST, c_last_name#36 ASC NULLS FIRST, c_preferred_cust_flag#37 ASC NULLS FIRST, c_birth_day#38 ASC NULLS FIRST, c_birth_month#39 ASC NULLS FIRST, c_birth_year#40 ASC NULLS FIRST, c_birth_country#41 ASC NULLS FIRST, c_login#42 ASC NULLS FIRST, c_email_address#43 ASC NULLS FIRST, c_last_review_date#44 ASC NULLS FIRST, ctr_total_return#17 ASC NULLS FIRST], [c_customer_id#32, c_salutation#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_day#38, c_birth_month#39, c_birth_year#40, c_birth_country#41, c_login#42, c_email_address#43, c_last_review_date#44, ctr_total_return#17] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q30/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q30/simplified.txt deleted file mode 100644 index f9ea8ad58..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q30/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (11) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - Filter [ctr_total_return] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returned_date_sk,wr_returning_addr_sk,wr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_returned_date_sk,wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) - Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] - InputAdapter - Exchange [ctr_state] #5 - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [wr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returned_date_sk,wr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_returned_date_sk,wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q31/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q31/explain.txt deleted file mode 100644 index 0ed0929fa..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q31/explain.txt +++ /dev/null @@ -1,563 +0,0 @@ -== Physical Plan == -* Sort (99) -+- Exchange (98) - +- * Project (97) - +- * BroadcastHashJoin Inner BuildRight (96) - :- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * BroadcastHashJoin Inner BuildRight (67) - : : :- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : :- * BroadcastHashJoin Inner BuildRight (35) - : : : : :- * HashAggregate (18) - : : : : : +- Exchange (17) - : : : : : +- * HashAggregate (16) - : : : : : +- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet default.date_dim (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet default.customer_address (10) - : : : : +- BroadcastExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- Exchange (32) - : : : : +- * HashAggregate (31) - : : : : +- * Project (30) - : : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : : :- * Project (27) - : : : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : : : :- * Filter (21) - : : : : : : +- * ColumnarToRow (20) - : : : : : : +- Scan parquet default.store_sales (19) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet default.date_dim (22) - : : : : +- ReusedExchange (28) - : : : +- BroadcastExchange (51) - : : : +- * HashAggregate (50) - : : : +- Exchange (49) - : : : +- * HashAggregate (48) - : : : +- * Project (47) - : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : :- * Project (44) - : : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : : :- * Filter (38) - : : : : : +- * ColumnarToRow (37) - : : : : : +- Scan parquet default.store_sales (36) - : : : : +- BroadcastExchange (42) - : : : : +- * Filter (41) - : : : : +- * ColumnarToRow (40) - : : : : +- Scan parquet default.date_dim (39) - : : : +- ReusedExchange (45) - : : +- BroadcastExchange (66) - : : +- * HashAggregate (65) - : : +- Exchange (64) - : : +- * HashAggregate (63) - : : +- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet default.web_sales (54) - : : : +- ReusedExchange (57) - : : +- ReusedExchange (60) - : +- BroadcastExchange (80) - : +- * HashAggregate (79) - : +- Exchange (78) - : +- * HashAggregate (77) - : +- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * Project (73) - : : +- * BroadcastHashJoin Inner BuildRight (72) - : : :- * Filter (70) - : : : +- * ColumnarToRow (69) - : : : +- Scan parquet default.web_sales (68) - : : +- ReusedExchange (71) - : +- ReusedExchange (74) - +- BroadcastExchange (95) - +- * HashAggregate (94) - +- Exchange (93) - +- * HashAggregate (92) - +- * Project (91) - +- * BroadcastHashJoin Inner BuildRight (90) - :- * Project (88) - : +- * BroadcastHashJoin Inner BuildRight (87) - : :- * Filter (85) - : : +- * ColumnarToRow (84) - : : +- Scan parquet default.web_sales (83) - : +- ReusedExchange (86) - +- ReusedExchange (89) - - -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#4, d_year#5, d_qoy#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] -Condition : ((((isnotnull(d_qoy#6) AND isnotnull(d_year#5)) AND (d_qoy#6 = 1)) AND (d_year#5 = 2000)) AND isnotnull(d_date_sk#4)) - -(7) BroadcastExchange -Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(9) Project [codegen id : 3] -Output [4]: [ss_addr_sk#2, ss_ext_sales_price#3, d_year#5, d_qoy#6] -Input [6]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, d_date_sk#4, d_year#5, d_qoy#6] - -(10) Scan parquet default.customer_address -Output [2]: [ca_address_sk#8, ca_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#8, ca_county#9] - -(12) Filter [codegen id : 2] -Input [2]: [ca_address_sk#8, ca_county#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) - -(13) BroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#8] -Join condition: None - -(15) Project [codegen id : 3] -Output [4]: [ss_ext_sales_price#3, d_year#5, d_qoy#6, ca_county#9] -Input [6]: [ss_addr_sk#2, ss_ext_sales_price#3, d_year#5, d_qoy#6, ca_address_sk#8, ca_county#9] - -(16) HashAggregate [codegen id : 3] -Input [4]: [ss_ext_sales_price#3, d_year#5, d_qoy#6, ca_county#9] -Keys [3]: [ca_county#9, d_qoy#6, d_year#5] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#11] -Results [4]: [ca_county#9, d_qoy#6, d_year#5, sum#12] - -(17) Exchange -Input [4]: [ca_county#9, d_qoy#6, d_year#5, sum#12] -Arguments: hashpartitioning(ca_county#9, d_qoy#6, d_year#5, 5), true, [id=#13] - -(18) HashAggregate [codegen id : 24] -Input [4]: [ca_county#9, d_qoy#6, d_year#5, sum#12] -Keys [3]: [ca_county#9, d_qoy#6, d_year#5] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#14] -Results [3]: [ca_county#9, d_year#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#14,17,2) AS store_sales#15] - -(19) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 6] -Input [3]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(21) Filter [codegen id : 6] -Input [3]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#2)) - -(22) Scan parquet default.date_dim -Output [3]: [d_date_sk#16, d_year#17, d_qoy#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(23) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#16, d_year#17, d_qoy#18] - -(24) Filter [codegen id : 4] -Input [3]: [d_date_sk#16, d_year#17, d_qoy#18] -Condition : ((((isnotnull(d_qoy#18) AND isnotnull(d_year#17)) AND (d_qoy#18 = 2)) AND (d_year#17 = 2000)) AND isnotnull(d_date_sk#16)) - -(25) BroadcastExchange -Input [3]: [d_date_sk#16, d_year#17, d_qoy#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#16] -Join condition: None - -(27) Project [codegen id : 6] -Output [4]: [ss_addr_sk#2, ss_ext_sales_price#3, d_year#17, d_qoy#18] -Input [6]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, d_date_sk#16, d_year#17, d_qoy#18] - -(28) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#20, ca_county#21] - -(29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#20] -Join condition: None - -(30) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#3, d_year#17, d_qoy#18, ca_county#21] -Input [6]: [ss_addr_sk#2, ss_ext_sales_price#3, d_year#17, d_qoy#18, ca_address_sk#20, ca_county#21] - -(31) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#3, d_year#17, d_qoy#18, ca_county#21] -Keys [3]: [ca_county#21, d_qoy#18, d_year#17] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#22] -Results [4]: [ca_county#21, d_qoy#18, d_year#17, sum#23] - -(32) Exchange -Input [4]: [ca_county#21, d_qoy#18, d_year#17, sum#23] -Arguments: hashpartitioning(ca_county#21, d_qoy#18, d_year#17, 5), true, [id=#24] - -(33) HashAggregate [codegen id : 7] -Input [4]: [ca_county#21, d_qoy#18, d_year#17, sum#23] -Keys [3]: [ca_county#21, d_qoy#18, d_year#17] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#25] -Results [2]: [ca_county#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#25,17,2) AS store_sales#26] - -(34) BroadcastExchange -Input [2]: [ca_county#21, store_sales#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#27] - -(35) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#21] -Join condition: None - -(36) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(37) ColumnarToRow [codegen id : 10] -Input [3]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(38) Filter [codegen id : 10] -Input [3]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#2)) - -(39) Scan parquet default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(40) ColumnarToRow [codegen id : 8] -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] - -(41) Filter [codegen id : 8] -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) - -(42) BroadcastExchange -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(43) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#28] -Join condition: None - -(44) Project [codegen id : 10] -Output [4]: [ss_addr_sk#2, ss_ext_sales_price#3, d_year#29, d_qoy#30] -Input [6]: [ss_sold_date_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, d_date_sk#28, d_year#29, d_qoy#30] - -(45) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#32, ca_county#33] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#32] -Join condition: None - -(47) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#3, d_year#29, d_qoy#30, ca_county#33] -Input [6]: [ss_addr_sk#2, ss_ext_sales_price#3, d_year#29, d_qoy#30, ca_address_sk#32, ca_county#33] - -(48) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#3, d_year#29, d_qoy#30, ca_county#33] -Keys [3]: [ca_county#33, d_qoy#30, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#34] -Results [4]: [ca_county#33, d_qoy#30, d_year#29, sum#35] - -(49) Exchange -Input [4]: [ca_county#33, d_qoy#30, d_year#29, sum#35] -Arguments: hashpartitioning(ca_county#33, d_qoy#30, d_year#29, 5), true, [id=#36] - -(50) HashAggregate [codegen id : 11] -Input [4]: [ca_county#33, d_qoy#30, d_year#29, sum#35] -Keys [3]: [ca_county#33, d_qoy#30, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#37] -Results [2]: [ca_county#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#37,17,2) AS store_sales#38] - -(51) BroadcastExchange -Input [2]: [ca_county#33, store_sales#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] - -(52) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#21] -Right keys [1]: [ca_county#33] -Join condition: None - -(53) Project [codegen id : 24] -Output [5]: [ca_county#9, d_year#5, store_sales#15, store_sales#26, store_sales#38] -Input [7]: [ca_county#9, d_year#5, store_sales#15, ca_county#21, store_sales#26, ca_county#33, store_sales#38] - -(54) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(55) ColumnarToRow [codegen id : 14] -Input [3]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42] - -(56) Filter [codegen id : 14] -Input [3]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42] -Condition : (isnotnull(ws_sold_date_sk#40) AND isnotnull(ws_bill_addr_sk#41)) - -(57) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#43, d_year#44, d_qoy#45] - -(58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#43] -Join condition: None - -(59) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#41, ws_ext_sales_price#42, d_year#44, d_qoy#45] -Input [6]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42, d_date_sk#43, d_year#44, d_qoy#45] - -(60) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#46, ca_county#47] - -(61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#41] -Right keys [1]: [ca_address_sk#46] -Join condition: None - -(62) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#42, d_year#44, d_qoy#45, ca_county#47] -Input [6]: [ws_bill_addr_sk#41, ws_ext_sales_price#42, d_year#44, d_qoy#45, ca_address_sk#46, ca_county#47] - -(63) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#42, d_year#44, d_qoy#45, ca_county#47] -Keys [3]: [ca_county#47, d_qoy#45, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#42))] -Aggregate Attributes [1]: [sum#48] -Results [4]: [ca_county#47, d_qoy#45, d_year#44, sum#49] - -(64) Exchange -Input [4]: [ca_county#47, d_qoy#45, d_year#44, sum#49] -Arguments: hashpartitioning(ca_county#47, d_qoy#45, d_year#44, 5), true, [id=#50] - -(65) HashAggregate [codegen id : 15] -Input [4]: [ca_county#47, d_qoy#45, d_year#44, sum#49] -Keys [3]: [ca_county#47, d_qoy#45, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#42))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#42))#51] -Results [2]: [ca_county#47, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#42))#51,17,2) AS web_sales#52] - -(66) BroadcastExchange -Input [2]: [ca_county#47, web_sales#52] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#53] - -(67) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#47] -Join condition: None - -(68) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(69) ColumnarToRow [codegen id : 18] -Input [3]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42] - -(70) Filter [codegen id : 18] -Input [3]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42] -Condition : (isnotnull(ws_sold_date_sk#40) AND isnotnull(ws_bill_addr_sk#41)) - -(71) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#54, d_year#55, d_qoy#56] - -(72) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#54] -Join condition: None - -(73) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#41, ws_ext_sales_price#42, d_year#55, d_qoy#56] -Input [6]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42, d_date_sk#54, d_year#55, d_qoy#56] - -(74) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#57, ca_county#58] - -(75) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#41] -Right keys [1]: [ca_address_sk#57] -Join condition: None - -(76) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#42, d_year#55, d_qoy#56, ca_county#58] -Input [6]: [ws_bill_addr_sk#41, ws_ext_sales_price#42, d_year#55, d_qoy#56, ca_address_sk#57, ca_county#58] - -(77) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#42, d_year#55, d_qoy#56, ca_county#58] -Keys [3]: [ca_county#58, d_qoy#56, d_year#55] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#42))] -Aggregate Attributes [1]: [sum#59] -Results [4]: [ca_county#58, d_qoy#56, d_year#55, sum#60] - -(78) Exchange -Input [4]: [ca_county#58, d_qoy#56, d_year#55, sum#60] -Arguments: hashpartitioning(ca_county#58, d_qoy#56, d_year#55, 5), true, [id=#61] - -(79) HashAggregate [codegen id : 19] -Input [4]: [ca_county#58, d_qoy#56, d_year#55, sum#60] -Keys [3]: [ca_county#58, d_qoy#56, d_year#55] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#42))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#42))#62] -Results [2]: [ca_county#58, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#42))#62,17,2) AS web_sales#63] - -(80) BroadcastExchange -Input [2]: [ca_county#58, web_sales#63] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#64] - -(81) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#47] -Right keys [1]: [ca_county#58] -Join condition: (CASE WHEN (web_sales#52 > 0.00) THEN CheckOverflow((promote_precision(web_sales#63) / promote_precision(web_sales#52)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#15 > 0.00) THEN CheckOverflow((promote_precision(store_sales#26) / promote_precision(store_sales#15)), DecimalType(37,20), true) ELSE null END) - -(82) Project [codegen id : 24] -Output [8]: [ca_county#9, d_year#5, store_sales#15, store_sales#26, store_sales#38, ca_county#47, web_sales#52, web_sales#63] -Input [9]: [ca_county#9, d_year#5, store_sales#15, store_sales#26, store_sales#38, ca_county#47, web_sales#52, ca_county#58, web_sales#63] - -(83) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(84) ColumnarToRow [codegen id : 22] -Input [3]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42] - -(85) Filter [codegen id : 22] -Input [3]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42] -Condition : (isnotnull(ws_sold_date_sk#40) AND isnotnull(ws_bill_addr_sk#41)) - -(86) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#65, d_year#66, d_qoy#67] - -(87) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#65] -Join condition: None - -(88) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#41, ws_ext_sales_price#42, d_year#66, d_qoy#67] -Input [6]: [ws_sold_date_sk#40, ws_bill_addr_sk#41, ws_ext_sales_price#42, d_date_sk#65, d_year#66, d_qoy#67] - -(89) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#68, ca_county#69] - -(90) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#41] -Right keys [1]: [ca_address_sk#68] -Join condition: None - -(91) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#42, d_year#66, d_qoy#67, ca_county#69] -Input [6]: [ws_bill_addr_sk#41, ws_ext_sales_price#42, d_year#66, d_qoy#67, ca_address_sk#68, ca_county#69] - -(92) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#42, d_year#66, d_qoy#67, ca_county#69] -Keys [3]: [ca_county#69, d_qoy#67, d_year#66] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#42))] -Aggregate Attributes [1]: [sum#70] -Results [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] - -(93) Exchange -Input [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] -Arguments: hashpartitioning(ca_county#69, d_qoy#67, d_year#66, 5), true, [id=#72] - -(94) HashAggregate [codegen id : 23] -Input [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] -Keys [3]: [ca_county#69, d_qoy#67, d_year#66] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#42))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#42))#73] -Results [2]: [ca_county#69, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#42))#73,17,2) AS web_sales#74] - -(95) BroadcastExchange -Input [2]: [ca_county#69, web_sales#74] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] - -(96) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#47] -Right keys [1]: [ca_county#69] -Join condition: (CASE WHEN (web_sales#63 > 0.00) THEN CheckOverflow((promote_precision(web_sales#74) / promote_precision(web_sales#63)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#26 > 0.00) THEN CheckOverflow((promote_precision(store_sales#38) / promote_precision(store_sales#26)), DecimalType(37,20), true) ELSE null END) - -(97) Project [codegen id : 24] -Output [6]: [ca_county#9, d_year#5, CheckOverflow((promote_precision(web_sales#63) / promote_precision(web_sales#52)), DecimalType(37,20), true) AS web_q1_q2_increase#76, CheckOverflow((promote_precision(store_sales#26) / promote_precision(store_sales#15)), DecimalType(37,20), true) AS store_q1_q2_increase#77, CheckOverflow((promote_precision(web_sales#74) / promote_precision(web_sales#63)), DecimalType(37,20), true) AS web_q2_q3_increase#78, CheckOverflow((promote_precision(store_sales#38) / promote_precision(store_sales#26)), DecimalType(37,20), true) AS store_q2_q3_increase#79] -Input [10]: [ca_county#9, d_year#5, store_sales#15, store_sales#26, store_sales#38, ca_county#47, web_sales#52, web_sales#63, ca_county#69, web_sales#74] - -(98) Exchange -Input [6]: [ca_county#9, d_year#5, web_q1_q2_increase#76, store_q1_q2_increase#77, web_q2_q3_increase#78, store_q2_q3_increase#79] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), true, [id=#80] - -(99) Sort [codegen id : 25] -Input [6]: [ca_county#9, d_year#5, web_q1_q2_increase#76, store_q1_q2_increase#77, web_q2_q3_increase#78, store_q2_q3_increase#79] -Arguments: [ca_county#9 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q31/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q31/simplified.txt deleted file mode 100644 index 5a5d61f9d..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q31/simplified.txt +++ /dev/null @@ -1,150 +0,0 @@ -WholeStageCodegen (25) - Sort [ca_county] - InputAdapter - Exchange [ca_county] #1 - WholeStageCodegen (24) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_addr_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [ca_address_sk,ca_county] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_addr_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_addr_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #14 - WholeStageCodegen (18) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #16 - WholeStageCodegen (22) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q32/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q32/explain.txt deleted file mode 100644 index 27f93fd7a..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q32/explain.txt +++ /dev/null @@ -1,175 +0,0 @@ -== Physical Plan == -CollectLimit (31) -+- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.catalog_sales (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.item (4) - : +- BroadcastExchange (25) - : +- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.catalog_sales (11) - : +- BroadcastExchange (18) - : +- * Project (17) - : +- * Filter (16) - : +- * ColumnarToRow (15) - : +- Scan parquet default.date_dim (14) - +- ReusedExchange (28) - - -(1) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_discount_amt#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_discount_amt#3] - -(3) Filter [codegen id : 6] -Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_discount_amt#3] -Condition : ((isnotnull(cs_item_sk#2) AND isnotnull(cs_ext_discount_amt#3)) AND isnotnull(cs_sold_date_sk#1)) - -(4) Scan parquet default.item -Output [2]: [i_item_sk#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#4, i_manufact_id#5] - -(6) Filter [codegen id : 1] -Input [2]: [i_item_sk#4, i_manufact_id#5] -Condition : ((isnotnull(i_manufact_id#5) AND (i_manufact_id#5 = 977)) AND isnotnull(i_item_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [i_item_sk#4] -Input [2]: [i_item_sk#4, i_manufact_id#5] - -(8) BroadcastExchange -Input [1]: [i_item_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] - -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#4] -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [cs_sold_date_sk#1, cs_ext_discount_amt#3, i_item_sk#4] -Input [4]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_discount_amt#3, i_item_sk#4] - -(11) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_discount_amt#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_discount_amt#3] - -(13) Filter [codegen id : 3] -Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_discount_amt#3] -Condition : (isnotnull(cs_sold_date_sk#1) AND isnotnull(cs_item_sk#2)) - -(14) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#7, d_date#8] - -(16) Filter [codegen id : 2] -Input [2]: [d_date_sk#7, d_date#8] -Condition : (((isnotnull(d_date#8) AND (d_date#8 >= 10983)) AND (d_date#8 <= 11073)) AND isnotnull(d_date_sk#7)) - -(17) Project [codegen id : 2] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] - -(18) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] - -(19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#7] -Join condition: None - -(20) Project [codegen id : 3] -Output [2]: [cs_item_sk#2, cs_ext_discount_amt#3] -Input [4]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_discount_amt#3, d_date_sk#7] - -(21) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#2, cs_ext_discount_amt#3] -Keys [1]: [cs_item_sk#2] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#3))] -Aggregate Attributes [2]: [sum#10, count#11] -Results [3]: [cs_item_sk#2, sum#12, count#13] - -(22) Exchange -Input [3]: [cs_item_sk#2, sum#12, count#13] -Arguments: hashpartitioning(cs_item_sk#2, 5), true, [id=#14] - -(23) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#2, sum#12, count#13] -Keys [1]: [cs_item_sk#2] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#3))#15] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#3))#15 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))#16, cs_item_sk#2 AS cs_item_sk#2#17] - -(24) Filter [codegen id : 4] -Input [2]: [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))#16, cs_item_sk#2#17] -Condition : isnotnull((CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))#16) - -(25) BroadcastExchange -Input [2]: [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))#16, cs_item_sk#2#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#18] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_sk#4] -Right keys [1]: [cs_item_sk#2#17] -Join condition: (cast(cs_ext_discount_amt#3 as decimal(14,7)) > (CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))#16) - -(27) Project [codegen id : 6] -Output [1]: [cs_sold_date_sk#1] -Input [5]: [cs_sold_date_sk#1, cs_ext_discount_amt#3, i_item_sk#4, (CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))#16, cs_item_sk#2#17] - -(28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#7] - -(29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#7] -Join condition: None - -(30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #19] -Input [2]: [cs_sold_date_sk#1, d_date_sk#7] - -(31) CollectLimit -Input [1]: [excess discount amount #19] -Arguments: 100 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q32/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q32/simplified.txt deleted file mode 100644 index 7e4b826cc..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q32/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -CollectLimit - WholeStageCodegen (6) - Project - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))] - Project [cs_sold_date_sk,cs_ext_discount_amt,i_item_sk] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_discount_amt] - InputAdapter - BroadcastExchange #1 - WholeStageCodegen (1) - Project [i_item_sk] - Filter [i_manufact_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (4) - Filter [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6))),cs_item_sk,sum,count] - InputAdapter - Exchange [cs_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] - Project [cs_item_sk,cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_discount_amt] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q33/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q33/explain.txt deleted file mode 100644 index 8d1558a01..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q33/explain.txt +++ /dev/null @@ -1,378 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) - :- * HashAggregate (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer_address (11) - : +- BroadcastExchange (27) - : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : :- * Filter (20) - : : +- * ColumnarToRow (19) - : : +- Scan parquet default.item (18) - : +- BroadcastExchange (25) - : +- * Project (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (41) - : : +- * BroadcastHashJoin Inner BuildRight (40) - : : :- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.catalog_sales (33) - : : : +- ReusedExchange (36) - : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) - - -(1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] - -(3) Filter [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_item_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_moy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -Condition : ((((isnotnull(d_year#6) AND isnotnull(d_moy#7)) AND (d_year#6 = 1998)) AND (d_moy#7 = 5)) AND isnotnull(d_date_sk#5)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] - -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(10) Project [codegen id : 5] -Output [3]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, d_date_sk#5] - -(11) Scan parquet default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] - -(13) Filter [codegen id : 2] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(14) Project [codegen id : 2] -Output [1]: [ca_address_sk#9] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] - -(15) BroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(17) Project [codegen id : 5] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#4] -Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] - -(18) Scan parquet default.item -Output [2]: [i_item_sk#12, i_manufact_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#12, i_manufact_id#13] - -(20) Filter [codegen id : 4] -Input [2]: [i_item_sk#12, i_manufact_id#13] -Condition : isnotnull(i_item_sk#12) - -(21) Scan parquet default.item -Output [2]: [i_category#14, i_manufact_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] -ReadSchema: struct - -(22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#14, i_manufact_id#13] - -(23) Filter [codegen id : 3] -Input [2]: [i_category#14, i_manufact_id#13] -Condition : (isnotnull(i_category#14) AND (i_category#14 = Electronics)) - -(24) Project [codegen id : 3] -Output [1]: [i_manufact_id#13 AS i_manufact_id#13#15] -Input [2]: [i_category#14, i_manufact_id#13] - -(25) BroadcastExchange -Input [1]: [i_manufact_id#13#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_manufact_id#13] -Right keys [1]: [i_manufact_id#13#15] -Join condition: None - -(27) BroadcastExchange -Input [2]: [i_item_sk#12, i_manufact_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] - -(28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(29) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#4, i_manufact_id#13] -Input [4]: [ss_item_sk#2, ss_ext_sales_price#4, i_item_sk#12, i_manufact_id#13] - -(30) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#4, i_manufact_id#13] -Keys [1]: [i_manufact_id#13] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [1]: [sum#18] -Results [2]: [i_manufact_id#13, sum#19] - -(31) Exchange -Input [2]: [i_manufact_id#13, sum#19] -Arguments: hashpartitioning(i_manufact_id#13, 5), true, [id=#20] - -(32) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#13, sum#19] -Keys [1]: [i_manufact_id#13] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#4))#21] -Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] - -(33) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] - -(35) Filter [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Condition : ((isnotnull(cs_sold_date_sk#23) AND isnotnull(cs_bill_addr_sk#24)) AND isnotnull(cs_item_sk#25)) - -(36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(38) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_date_sk#5] - -(39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#9] - -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(41) Project [codegen id : 11] -Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#9] - -(42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#12, i_manufact_id#13] - -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#13] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#12, i_manufact_id#13] - -(45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#13] -Keys [1]: [i_manufact_id#13] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_manufact_id#13, sum#28] - -(46) Exchange -Input [2]: [i_manufact_id#13, sum#28] -Arguments: hashpartitioning(i_manufact_id#13, 5), true, [id=#29] - -(47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#13, sum#28] -Keys [1]: [i_manufact_id#13] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#30] -Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] - -(48) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] - -(50) Filter [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Condition : ((isnotnull(ws_sold_date_sk#32) AND isnotnull(ws_bill_addr_sk#34)) AND isnotnull(ws_item_sk#33)) - -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, d_date_sk#5] - -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#9] - -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#9] - -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#12, i_manufact_id#13] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#13] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#12, i_manufact_id#13] - -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#13] -Keys [1]: [i_manufact_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#36] -Results [2]: [i_manufact_id#13, sum#37] - -(61) Exchange -Input [2]: [i_manufact_id#13, sum#37] -Arguments: hashpartitioning(i_manufact_id#13, 5), true, [id=#38] - -(62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#13, sum#37] -Keys [1]: [i_manufact_id#13] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#39] -Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] - -(63) Union - -(64) HashAggregate [codegen id : 19] -Input [2]: [i_manufact_id#13, total_sales#22] -Keys [1]: [i_manufact_id#13] -Functions [1]: [partial_sum(total_sales#22)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_manufact_id#13, sum#43, isEmpty#44] - -(65) Exchange -Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#13, 5), true, [id=#45] - -(66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] -Keys [1]: [i_manufact_id#13] -Functions [1]: [sum(total_sales#22)] -Aggregate Attributes [1]: [sum(total_sales#22)#46] -Results [2]: [i_manufact_id#13, sum(total_sales#22)#46 AS total_sales#47] - -(67) TakeOrderedAndProject -Input [2]: [i_manufact_id#13, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#13, total_sales#47] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q33/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q33/simplified.txt deleted file mode 100644 index 14787f0bb..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q33/simplified.txt +++ /dev/null @@ -1,101 +0,0 @@ -TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (20) - HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_manufact_id] #1 - WholeStageCodegen (19) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #2 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_category,i_manufact_id] - WholeStageCodegen (12) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #7 - WholeStageCodegen (11) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 - WholeStageCodegen (18) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #8 - WholeStageCodegen (17) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q34/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q34/explain.txt deleted file mode 100644 index 18f465cae..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q34/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -* Sort (36) -+- Exchange (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Filter (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.store (11) - : +- BroadcastExchange (22) - : +- * Project (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.household_demographics (18) - +- BroadcastExchange (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.customer (29) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5] -Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_hdemo_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_dom#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#7, d_dom#8] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#7, d_dom#8] -Condition : (((((d_dom#8 >= 1) AND (d_dom#8 <= 3)) OR ((d_dom#8 >= 25) AND (d_dom#8 <= 28))) AND d_year#7 IN (1999,2000,2001)) AND isnotnull(d_date_sk#6)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#6] -Input [3]: [d_date_sk#6, d_year#7, d_dom#8] - -(8) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#6] -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5] -Input [6]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5, d_date_sk#6] - -(11) Scan parquet default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#10, s_county#11] - -(13) Filter [codegen id : 2] -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(14) Project [codegen id : 2] -Output [1]: [s_store_sk#10] -Input [2]: [s_store_sk#10, s_county#11] - -(15) BroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#10] -Join condition: None - -(17) Project [codegen id : 4] -Output [3]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_ticket_number#5] -Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5, s_store_sk#10] - -(18) Scan parquet default.household_demographics -Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_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] -Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] - -(20) Filter [codegen id : 3] -Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = >10000) OR (hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND (CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) ELSE null END > 1.2)) AND isnotnull(hd_demo_sk#13)) - -(21) Project [codegen id : 3] -Output [1]: [hd_demo_sk#13] -Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] - -(22) BroadcastExchange -Input [1]: [hd_demo_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#13] -Join condition: None - -(24) Project [codegen id : 4] -Output [2]: [ss_customer_sk#2, ss_ticket_number#5] -Input [4]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_ticket_number#5, hd_demo_sk#13] - -(25) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#2, ss_ticket_number#5] -Keys [2]: [ss_ticket_number#5, ss_customer_sk#2] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] - -(26) Exchange -Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, [id=#20] - -(27) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] -Keys [2]: [ss_ticket_number#5, ss_customer_sk#2] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#21] -Results [3]: [ss_ticket_number#5, ss_customer_sk#2, count(1)#21 AS cnt#22] - -(28) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22] -Condition : ((cnt#22 >= 15) AND (cnt#22 <= 20)) - -(29) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27] - -(31) Filter [codegen id : 5] -Input [5]: [c_customer_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27] -Condition : isnotnull(c_customer_sk#23) - -(32) BroadcastExchange -Input [5]: [c_customer_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] - -(33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#23] -Join condition: None - -(34) Project [codegen id : 6] -Output [6]: [c_last_name#26, c_first_name#25, c_salutation#24, c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22] -Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, c_customer_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27] - -(35) Exchange -Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22] -Arguments: rangepartitioning(c_last_name#26 ASC NULLS FIRST, c_first_name#25 ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_preferred_cust_flag#27 DESC NULLS LAST, 5), true, [id=#29] - -(36) Sort [codegen id : 7] -Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22] -Arguments: [c_last_name#26 ASC NULLS FIRST, c_first_name#25 ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_preferred_cust_flag#27 DESC NULLS LAST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q34/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q34/simplified.txt deleted file mode 100644 index 5af07f1d4..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q34/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (7) - Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] - InputAdapter - Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - InputAdapter - Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_dom,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [s_store_sk] - Filter [s_county,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q35/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q35/explain.txt deleted file mode 100644 index 5370b6872..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q35/explain.txt +++ /dev/null @@ -1,274 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (33) - : : +- * Filter (32) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (31) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (23) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (19) - : : +- BroadcastExchange (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Filter (26) - : : : +- * ColumnarToRow (25) - : : : +- Scan parquet default.catalog_sales (24) - : : +- ReusedExchange (27) - : +- BroadcastExchange (37) - : +- * Filter (36) - : +- * ColumnarToRow (35) - : +- Scan parquet default.customer_address (34) - +- BroadcastExchange (43) - +- * Filter (42) - +- * ColumnarToRow (41) - +- Scan parquet default.customer_demographics (40) - - -(1) Scan parquet default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(3) Filter [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(4) Scan parquet default.store_sales -Output [2]: [ss_sold_date_sk#6, ss_customer_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_sold_date_sk#6, ss_customer_sk#7] - -(6) Filter [codegen id : 2] -Input [2]: [ss_sold_date_sk#6, ss_customer_sk#7] -Condition : isnotnull(ss_sold_date_sk#6) - -(7) Scan parquet default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] - -(9) Filter [codegen id : 1] -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : ((((isnotnull(d_year#9) AND isnotnull(d_qoy#10)) AND (d_year#9 = 2002)) AND (d_qoy#10 < 4)) AND isnotnull(d_date_sk#8)) - -(10) Project [codegen id : 1] -Output [1]: [d_date_sk#8] -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] - -(11) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(13) Project [codegen id : 2] -Output [1]: [ss_customer_sk#7] -Input [3]: [ss_sold_date_sk#6, ss_customer_sk#7, d_date_sk#8] - -(14) BroadcastExchange -Input [1]: [ss_customer_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(15) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#7] -Join condition: None - -(16) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#13, ws_bill_customer_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 4] -Input [2]: [ws_sold_date_sk#13, ws_bill_customer_sk#14] - -(18) Filter [codegen id : 4] -Input [2]: [ws_sold_date_sk#13, ws_bill_customer_sk#14] -Condition : isnotnull(ws_sold_date_sk#13) - -(19) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#8] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(21) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_sold_date_sk#13, ws_bill_customer_sk#14, d_date_sk#8] - -(22) BroadcastExchange -Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#14] -Join condition: None - -(24) Scan parquet default.catalog_sales -Output [2]: [cs_sold_date_sk#16, cs_ship_customer_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 6] -Input [2]: [cs_sold_date_sk#16, cs_ship_customer_sk#17] - -(26) Filter [codegen id : 6] -Input [2]: [cs_sold_date_sk#16, cs_ship_customer_sk#17] -Condition : isnotnull(cs_sold_date_sk#16) - -(27) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#8] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(29) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_sold_date_sk#16, cs_ship_customer_sk#17, d_date_sk#8] - -(30) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] - -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] -Join condition: None - -(32) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(34) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_state#20] - -(36) Filter [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : isnotnull(ca_address_sk#19) - -(37) BroadcastExchange -Input [2]: [ca_address_sk#19, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#19] -Join condition: None - -(39) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#20] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#19, ca_state#20] - -(40) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] - -(42) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Condition : isnotnull(cd_demo_sk#22) - -(43) BroadcastExchange -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#22] -Join condition: None - -(45) Project [codegen id : 9] -Output [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] - -(46) HashAggregate [codegen id : 9] -Input [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#25), partial_max(cd_dep_count#25), partial_avg(cast(cd_dep_count#25 as bigint)), partial_min(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_avg(cast(cd_dep_employed_count#26 as bigint)), partial_min(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_avg(cast(cd_dep_college_count#27 as bigint))] -Aggregate Attributes [13]: [count#29, min#30, max#31, sum#32, count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41] -Results [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] - -(47) Exchange -Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), true, [id=#55] - -(48) HashAggregate [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] -Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [count(1), min(cd_dep_count#25), max(cd_dep_count#25), avg(cast(cd_dep_count#25 as bigint)), min(cd_dep_employed_count#26), max(cd_dep_employed_count#26), avg(cast(cd_dep_employed_count#26 as bigint)), min(cd_dep_college_count#27), max(cd_dep_college_count#27), avg(cast(cd_dep_college_count#27 as bigint))] -Aggregate Attributes [10]: [count(1)#56, min(cd_dep_count#25)#57, max(cd_dep_count#25)#58, avg(cast(cd_dep_count#25 as bigint))#59, min(cd_dep_employed_count#26)#60, max(cd_dep_employed_count#26)#61, avg(cast(cd_dep_employed_count#26 as bigint))#62, min(cd_dep_college_count#27)#63, max(cd_dep_college_count#27)#64, avg(cast(cd_dep_college_count#27 as bigint))#65] -Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, count(1)#56 AS cnt1#66, min(cd_dep_count#25)#57 AS min(cd_dep_count)#67, max(cd_dep_count#25)#58 AS max(cd_dep_count)#68, avg(cast(cd_dep_count#25 as bigint))#59 AS avg(cd_dep_count)#69, cd_dep_employed_count#26, count(1)#56 AS cnt2#70, min(cd_dep_employed_count#26)#60 AS min(cd_dep_employed_count)#71, max(cd_dep_employed_count#26)#61 AS max(cd_dep_employed_count)#72, avg(cast(cd_dep_employed_count#26 as bigint))#62 AS avg(cd_dep_employed_count)#73, cd_dep_college_count#27, count(1)#56 AS cnt3#74, min(cd_dep_college_count#27)#63 AS min(cd_dep_college_count)#75, max(cd_dep_college_count#27)#64 AS max(cd_dep_college_count)#76, avg(cast(cd_dep_college_count#27 as bigint))#65 AS avg(cd_dep_college_count)#77, cd_dep_count#25 AS aggOrder#78] - -(49) TakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cnt1#66, min(cd_dep_count)#67, max(cd_dep_count)#68, avg(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, min(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, avg(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, min(cd_dep_college_count)#75, max(cd_dep_college_count)#76, avg(cd_dep_college_count)#77, aggOrder#78] -Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, aggOrder#78 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cnt1#66, min(cd_dep_count)#67, max(cd_dep_count)#68, avg(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, min(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, avg(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, min(cd_dep_college_count)#75, max(cd_dep_college_count)#76, avg(cd_dep_college_count)#77] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q35/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q35/simplified.txt deleted file mode 100644 index 9f27114f7..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q35/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cast(cd_dep_count as bigint)),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cast(cd_dep_employed_count as bigint)),min(cd_dep_college_count),max(cd_dep_college_count),avg(cast(cd_dep_college_count as bigint)),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),aggOrder,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - InputAdapter - Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q36/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q36/explain.txt deleted file mode 100644 index 73174b735..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q36/explain.txt +++ /dev/null @@ -1,180 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- Window (30) - +- * Sort (29) - +- Exchange (28) - +- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Expand (24) - +- * Project (23) - +- * BroadcastHashJoin Inner BuildRight (22) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.date_dim (4) - : +- BroadcastExchange (14) - : +- * Filter (13) - : +- * ColumnarToRow (12) - : +- Scan parquet default.item (11) - +- BroadcastExchange (21) - +- * Project (20) - +- * Filter (19) - +- * ColumnarToRow (18) - +- Scan parquet default.store (17) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_net_profit#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_net_profit#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_net_profit#5] -Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#7] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk#6)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#7] - -(8) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#6] -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [ss_item_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_net_profit#5] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_net_profit#5, d_date_sk#6] - -(11) Scan parquet default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#9, i_class#10, i_category#11] - -(13) Filter [codegen id : 2] -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(14) BroadcastExchange -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#9] -Join condition: None - -(16) Project [codegen id : 4] -Output [5]: [ss_store_sk#3, ss_ext_sales_price#4, ss_net_profit#5, i_class#10, i_category#11] -Input [7]: [ss_item_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_net_profit#5, i_item_sk#9, i_class#10, i_category#11] - -(17) Scan parquet default.store -Output [2]: [s_store_sk#13, s_state#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#13, s_state#14] - -(19) Filter [codegen id : 3] -Input [2]: [s_store_sk#13, s_state#14] -Condition : ((isnotnull(s_state#14) AND (s_state#14 = TN)) AND isnotnull(s_store_sk#13)) - -(20) Project [codegen id : 3] -Output [1]: [s_store_sk#13] -Input [2]: [s_store_sk#13, s_state#14] - -(21) BroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] -Join condition: None - -(23) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#4, ss_net_profit#5, i_category#11, i_class#10] -Input [6]: [ss_store_sk#3, ss_ext_sales_price#4, ss_net_profit#5, i_class#10, i_category#11, s_store_sk#13] - -(24) Expand [codegen id : 4] -Input [4]: [ss_ext_sales_price#4, ss_net_profit#5, i_category#11, i_class#10] -Arguments: [List(ss_ext_sales_price#4, ss_net_profit#5, i_category#11, i_class#10, 0), List(ss_ext_sales_price#4, ss_net_profit#5, i_category#11, null, 1), List(ss_ext_sales_price#4, ss_net_profit#5, null, null, 3)], [ss_ext_sales_price#4, ss_net_profit#5, i_category#16, i_class#17, spark_grouping_id#18] - -(25) HashAggregate [codegen id : 4] -Input [5]: [ss_ext_sales_price#4, ss_net_profit#5, i_category#16, i_class#17, spark_grouping_id#18] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [2]: [sum#19, sum#20] -Results [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#21, sum#22] - -(26) Exchange -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#21, sum#22] -Arguments: hashpartitioning(i_category#16, i_class#17, spark_grouping_id#18, 5), true, [id=#23] - -(27) HashAggregate [codegen id : 5] -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#21, sum#22] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#5))#24, sum(UnscaledValue(ss_ext_sales_price#4))#25] -Results [7]: [CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#24,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#25,17,2))), DecimalType(37,20), true) AS gross_margin#26, i_category#16, i_class#17, (cast((shiftright(spark_grouping_id#18, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#18, 0) & 1) as tinyint)) AS lochierarchy#27, (cast((shiftright(spark_grouping_id#18, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#18, 0) & 1) as tinyint)) AS _w1#28, CASE WHEN (cast(cast((shiftright(spark_grouping_id#18, 0) & 1) as tinyint) as int) = 0) THEN i_category#16 END AS _w2#29, CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#24,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#25,17,2))), DecimalType(37,20), true) AS _w3#30] - -(28) Exchange -Input [7]: [gross_margin#26, i_category#16, i_class#17, lochierarchy#27, _w1#28, _w2#29, _w3#30] -Arguments: hashpartitioning(_w1#28, _w2#29, 5), true, [id=#31] - -(29) Sort [codegen id : 6] -Input [7]: [gross_margin#26, i_category#16, i_class#17, lochierarchy#27, _w1#28, _w2#29, _w3#30] -Arguments: [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w3#30 ASC NULLS FIRST], false, 0 - -(30) Window -Input [7]: [gross_margin#26, i_category#16, i_class#17, lochierarchy#27, _w1#28, _w2#29, _w3#30] -Arguments: [rank(_w3#30) windowspecdefinition(_w1#28, _w2#29, _w3#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#32], [_w1#28, _w2#29], [_w3#30 ASC NULLS FIRST] - -(31) Project [codegen id : 7] -Output [5]: [gross_margin#26, i_category#16, i_class#17, lochierarchy#27, rank_within_parent#32] -Input [8]: [gross_margin#26, i_category#16, i_class#17, lochierarchy#27, _w1#28, _w2#29, _w3#30, rank_within_parent#32] - -(32) TakeOrderedAndProject -Input [5]: [gross_margin#26, i_category#16, i_class#17, lochierarchy#27, rank_within_parent#32] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (cast(lochierarchy#27 as int) = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#32 ASC NULLS FIRST], [gross_margin#26, i_category#16, i_class#17, lochierarchy#27, rank_within_parent#32] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q36/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q36/simplified.txt deleted file mode 100644 index d4a081452..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q36/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (7) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w3,_w1,_w2] - WholeStageCodegen (6) - Sort [_w1,_w2,_w3] - InputAdapter - Exchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w1,_w2,_w3,sum,sum] - InputAdapter - Exchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk] - Filter [s_state,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_state] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q37/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q37/explain.txt deleted file mode 100644 index 896d6c571..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q37/explain.txt +++ /dev/null @@ -1,160 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * BroadcastHashJoin Inner BuildRight (23) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.item (1) - : : +- BroadcastExchange (9) - : : +- * Project (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet default.inventory (5) - : +- BroadcastExchange (16) - : +- * Project (15) - : +- * Filter (14) - : +- * ColumnarToRow (13) - : +- Scan parquet default.date_dim (12) - +- BroadcastExchange (22) - +- * Filter (21) - +- * ColumnarToRow (20) - +- Scan parquet default.catalog_sales (19) - - -(1) Scan parquet default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), In(i_manufact_id, [677,940,694,808]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] - -(3) Filter [codegen id : 4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (cast(i_current_price#4 as decimal(12,2)) <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) - -(4) Project [codegen id : 4] -Output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] - -(5) Scan parquet default.inventory -Output [3]: [inv_date_sk#6, inv_item_sk#7, inv_quantity_on_hand#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk), IsNotNull(inv_date_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [inv_date_sk#6, inv_item_sk#7, inv_quantity_on_hand#8] - -(7) Filter [codegen id : 1] -Input [3]: [inv_date_sk#6, inv_item_sk#7, inv_quantity_on_hand#8] -Condition : ((((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) AND isnotnull(inv_date_sk#6)) - -(8) Project [codegen id : 1] -Output [2]: [inv_date_sk#6, inv_item_sk#7] -Input [3]: [inv_date_sk#6, inv_item_sk#7, inv_quantity_on_hand#8] - -(9) BroadcastExchange -Input [2]: [inv_date_sk#6, inv_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#9] - -(10) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#7] -Join condition: None - -(11) Project [codegen id : 4] -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#6] -Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#6, inv_item_sk#7] - -(12) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_date#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#10, d_date#11] - -(14) Filter [codegen id : 2] -Input [2]: [d_date_sk#10, d_date#11] -Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 10988)) AND (d_date#11 <= 11048)) AND isnotnull(d_date_sk#10)) - -(15) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_date#11] - -(16) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#6] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(18) Project [codegen id : 4] -Output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#6, d_date_sk#10] - -(19) Scan parquet default.catalog_sales -Output [1]: [cs_item_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 3] -Input [1]: [cs_item_sk#13] - -(21) Filter [codegen id : 3] -Input [1]: [cs_item_sk#13] -Condition : isnotnull(cs_item_sk#13) - -(22) BroadcastExchange -Input [1]: [cs_item_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#13] -Join condition: None - -(24) Project [codegen id : 4] -Output [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, cs_item_sk#13] - -(25) HashAggregate [codegen id : 4] -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] - -(26) Exchange -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), true, [id=#15] - -(27) HashAggregate [codegen id : 5] -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] - -(28) TakeOrderedAndProject -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q37/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q37/simplified.txt deleted file mode 100644 index 6d3216fff..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q37/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - WholeStageCodegen (5) - HashAggregate [i_item_id,i_item_desc,i_current_price] - InputAdapter - Exchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - Filter [i_current_price,i_manufact_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [inv_date_sk,inv_item_sk] - Filter [inv_quantity_on_hand,inv_item_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q38/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q38/explain.txt deleted file mode 100644 index 74454cf32..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q38/explain.txt +++ /dev/null @@ -1,323 +0,0 @@ -== Physical Plan == -* HashAggregate (54) -+- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * BroadcastHashJoin LeftSemi BuildRight (44) - :- * BroadcastHashJoin LeftSemi BuildRight (30) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer (11) - : +- BroadcastExchange (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) - - -(1) Scan parquet default.store_sales -Output [2]: [ss_sold_date_sk#1, ss_customer_sk#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 11] -Input [2]: [ss_sold_date_sk#1, ss_customer_sk#2] - -(3) Filter [codegen id : 11] -Input [2]: [ss_sold_date_sk#1, ss_customer_sk#2] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_customer_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#3, d_date#4, d_month_seq#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#3, d_date#4, d_month_seq#5] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#3, d_date#4, d_month_seq#5] -Condition : (((isnotnull(d_month_seq#5) AND (d_month_seq#5 >= 1200)) AND (d_month_seq#5 <= 1211)) AND isnotnull(d_date_sk#3)) - -(7) Project [codegen id : 1] -Output [2]: [d_date_sk#3, d_date#4] -Input [3]: [d_date_sk#3, d_date#4, d_month_seq#5] - -(8) BroadcastExchange -Input [2]: [d_date_sk#3, d_date#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] - -(9) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#3] -Join condition: None - -(10) Project [codegen id : 11] -Output [2]: [ss_customer_sk#2, d_date#4] -Input [4]: [ss_sold_date_sk#1, ss_customer_sk#2, d_date_sk#3, d_date#4] - -(11) Scan parquet default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] - -(13) Filter [codegen id : 2] -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(14) BroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(15) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#7] -Join condition: None - -(16) Project [codegen id : 11] -Output [3]: [d_date#4, c_first_name#8, c_last_name#9] -Input [5]: [ss_customer_sk#2, d_date#4, c_customer_sk#7, c_first_name#8, c_last_name#9] - -(17) Scan parquet default.catalog_sales -Output [2]: [cs_sold_date_sk#11, cs_bill_customer_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 5] -Input [2]: [cs_sold_date_sk#11, cs_bill_customer_sk#12] - -(19) Filter [codegen id : 5] -Input [2]: [cs_sold_date_sk#11, cs_bill_customer_sk#12] -Condition : (isnotnull(cs_sold_date_sk#11) AND isnotnull(cs_bill_customer_sk#12)) - -(20) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#13, d_date#14] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(22) Project [codegen id : 5] -Output [2]: [cs_bill_customer_sk#12, d_date#14] -Input [4]: [cs_sold_date_sk#11, cs_bill_customer_sk#12, d_date_sk#13, d_date#14] - -(23) ReusedExchange [Reuses operator id: 14] -Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_bill_customer_sk#12] -Right keys [1]: [c_customer_sk#15] -Join condition: None - -(25) Project [codegen id : 5] -Output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Input [5]: [cs_bill_customer_sk#12, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] - -(26) HashAggregate [codegen id : 5] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(27) Exchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), true, [id=#18] - -(28) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(29) BroadcastExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -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=#19] - -(30) BroadcastHashJoin [codegen id : 11] -Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#4, 0), isnull(d_date#4)] -Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 0), isnull(d_date#14)] -Join condition: None - -(31) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#20, ws_bill_customer_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) ColumnarToRow [codegen id : 9] -Input [2]: [ws_sold_date_sk#20, ws_bill_customer_sk#21] - -(33) Filter [codegen id : 9] -Input [2]: [ws_sold_date_sk#20, ws_bill_customer_sk#21] -Condition : (isnotnull(ws_sold_date_sk#20) AND isnotnull(ws_bill_customer_sk#21)) - -(34) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#22, d_date#23] - -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join condition: None - -(36) Project [codegen id : 9] -Output [2]: [ws_bill_customer_sk#21, d_date#23] -Input [4]: [ws_sold_date_sk#20, ws_bill_customer_sk#21, d_date_sk#22, d_date#23] - -(37) ReusedExchange [Reuses operator id: 14] -Output [3]: [c_customer_sk#24, c_first_name#25, c_last_name#26] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_bill_customer_sk#21] -Right keys [1]: [c_customer_sk#24] -Join condition: None - -(39) Project [codegen id : 9] -Output [3]: [c_last_name#26, c_first_name#25, d_date#23] -Input [5]: [ws_bill_customer_sk#21, d_date#23, c_customer_sk#24, c_first_name#25, c_last_name#26] - -(40) HashAggregate [codegen id : 9] -Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -Keys [3]: [c_last_name#26, c_first_name#25, d_date#23] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#26, c_first_name#25, d_date#23] - -(41) Exchange -Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -Arguments: hashpartitioning(c_last_name#26, c_first_name#25, d_date#23, 5), true, [id=#27] - -(42) HashAggregate [codegen id : 10] -Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -Keys [3]: [c_last_name#26, c_first_name#25, d_date#23] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#26, c_first_name#25, d_date#23] - -(43) BroadcastExchange -Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -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=#28] - -(44) BroadcastHashJoin [codegen id : 11] -Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#4, 0), isnull(d_date#4)] -Right keys [6]: [coalesce(c_last_name#26, ), isnull(c_last_name#26), coalesce(c_first_name#25, ), isnull(c_first_name#25), coalesce(d_date#23, 0), isnull(d_date#23)] -Join condition: None - -(45) HashAggregate [codegen id : 11] -Input [3]: [d_date#4, c_first_name#8, c_last_name#9] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(46) Exchange -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#4, 5), true, [id=#29] - -(47) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(48) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(49) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(50) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(51) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results: [] - -(52) HashAggregate [codegen id : 12] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [1]: [count#31] - -(53) Exchange -Input [1]: [count#31] -Arguments: SinglePartition, true, [id=#32] - -(54) HashAggregate [codegen id : 13] -Input [1]: [count#31] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [1]: [count(1)#33 AS count(1)#34] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q38/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q38/simplified.txt deleted file mode 100644 index a5b57a4ac..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q38/simplified.txt +++ /dev/null @@ -1,80 +0,0 @@ -WholeStageCodegen (13) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (12) - HashAggregate [count,count] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (11) - HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - Project [d_date,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39a/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39a/explain.txt deleted file mode 100644 index b2cc849c6..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39a/explain.txt +++ /dev/null @@ -1,292 +0,0 @@ -== Physical Plan == -* Sort (52) -+- Exchange (51) - +- * BroadcastHashJoin Inner BuildRight (50) - :- * Project (27) - : +- * Filter (26) - : +- * HashAggregate (25) - : +- Exchange (24) - : +- * HashAggregate (23) - : +- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.warehouse (10) - : +- BroadcastExchange (20) - : +- * Project (19) - : +- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet default.date_dim (16) - +- BroadcastExchange (49) - +- * Project (48) - +- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet default.inventory (28) - : : +- ReusedExchange (31) - : +- ReusedExchange (34) - +- BroadcastExchange (41) - +- * Project (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet default.date_dim (37) - - -(1) Scan parquet default.inventory -Output [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Condition : ((isnotnull(inv_item_sk#2) AND isnotnull(inv_warehouse_sk#3)) AND isnotnull(inv_date_sk#1)) - -(4) Scan parquet default.item -Output [1]: [i_item_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#5] - -(6) Filter [codegen id : 1] -Input [1]: [i_item_sk#5] -Condition : isnotnull(i_item_sk#5) - -(7) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#2] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_date_sk#1, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#5] -Input [5]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#5] - -(10) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(12) Filter [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#9] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#7] -Join condition: None - -(15) Project [codegen id : 4] -Output [5]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8] -Input [6]: [inv_date_sk#1, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8] - -(16) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_moy#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] - -(18) Filter [codegen id : 3] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 2001)) AND (d_moy#12 = 1)) AND isnotnull(d_date_sk#10)) - -(19) Project [codegen id : 3] -Output [2]: [d_date_sk#10, d_moy#12] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] - -(20) BroadcastExchange -Input [2]: [d_date_sk#10, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#1] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(22) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8, d_moy#12] -Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#10, d_moy#12] - -(23) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8, d_moy#12] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [5]: [n#14, avg#15, m2#16, sum#17, count#18] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] - -(24) Exchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, 5), true, [id=#24] - -(25) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stddev_samp(cast(inv_quantity_on_hand#4 as double))#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] - -(26) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stdev#27, mean#28] -Condition : (CASE WHEN (mean#28 = 0.0) THEN 0.0 ELSE (stdev#27 / mean#28) END > 1.0) - -(27) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, mean#28, CASE WHEN (mean#28 = 0.0) THEN null ELSE (stdev#27 / mean#28) END AS cov#29] -Input [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stdev#27, mean#28] - -(28) Scan parquet default.inventory -Output [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 8] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] - -(30) Filter [codegen id : 8] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Condition : ((isnotnull(inv_item_sk#2) AND isnotnull(inv_warehouse_sk#3)) AND isnotnull(inv_date_sk#1)) - -(31) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#30] - -(32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#2] -Right keys [1]: [i_item_sk#30] -Join condition: None - -(33) Project [codegen id : 8] -Output [4]: [inv_date_sk#1, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#30] -Input [5]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#30] - -(34) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#31, w_warehouse_name#32] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#31] -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32] -Input [6]: [inv_date_sk#1, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32] - -(37) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_moy#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] - -(39) Filter [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] -Condition : ((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#33)) - -(40) Project [codegen id : 7] -Output [2]: [d_date_sk#33, d_moy#35] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] - -(41) BroadcastExchange -Input [2]: [d_date_sk#33, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] - -(42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#1] -Right keys [1]: [d_date_sk#33] -Join condition: None - -(43) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32, d_moy#35] -Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32, d_date_sk#33, d_moy#35] - -(44) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32, d_moy#35] -Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [5]: [n#37, avg#38, m2#39, sum#40, count#41] -Results [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] - -(45) Exchange -Input [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] -Arguments: hashpartitioning(w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, 5), true, [id=#47] - -(46) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] -Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#48, avg(cast(inv_quantity_on_hand#4 as bigint))#49] -Results [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#4 as double))#48 AS stdev#50, avg(cast(inv_quantity_on_hand#4 as bigint))#49 AS mean#51] - -(47) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stdev#50, mean#51] -Condition : (CASE WHEN (mean#51 = 0.0) THEN 0.0 ELSE (stdev#50 / mean#51) END > 1.0) - -(48) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, mean#51, CASE WHEN (mean#51 = 0.0) THEN null ELSE (stdev#50 / mean#51) END AS cov#52] -Input [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stdev#50, mean#51] - -(49) BroadcastExchange -Input [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, mean#51, cov#52] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#53] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [i_item_sk#5, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#30, w_warehouse_sk#31] -Join condition: None - -(51) Exchange -Input [10]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, mean#28, cov#29, w_warehouse_sk#31, i_item_sk#30, d_moy#35, mean#51, cov#52] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#5 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#28 ASC NULLS FIRST, cov#29 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#51 ASC NULLS FIRST, cov#52 ASC NULLS FIRST, 5), true, [id=#54] - -(52) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, mean#28, cov#29, w_warehouse_sk#31, i_item_sk#30, d_moy#35, mean#51, cov#52] -Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#5 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#28 ASC NULLS FIRST, cov#29 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#51 ASC NULLS FIRST, cov#52 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39a/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39a/simplified.txt deleted file mode 100644 index f4e23c837..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39a/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -WholeStageCodegen (11) - Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] - InputAdapter - Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_date_sk,inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_date_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_date_sk,d_moy] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_date_sk,inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_date_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - InputAdapter - ReusedExchange [i_item_sk] #3 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - Project [d_date_sk,d_moy] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39b/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39b/explain.txt deleted file mode 100644 index 92c2d5ed4..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39b/explain.txt +++ /dev/null @@ -1,292 +0,0 @@ -== Physical Plan == -* Sort (52) -+- Exchange (51) - +- * BroadcastHashJoin Inner BuildRight (50) - :- * Project (27) - : +- * Filter (26) - : +- * HashAggregate (25) - : +- Exchange (24) - : +- * HashAggregate (23) - : +- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.warehouse (10) - : +- BroadcastExchange (20) - : +- * Project (19) - : +- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet default.date_dim (16) - +- BroadcastExchange (49) - +- * Project (48) - +- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet default.inventory (28) - : : +- ReusedExchange (31) - : +- ReusedExchange (34) - +- BroadcastExchange (41) - +- * Project (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet default.date_dim (37) - - -(1) Scan parquet default.inventory -Output [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Condition : ((isnotnull(inv_item_sk#2) AND isnotnull(inv_warehouse_sk#3)) AND isnotnull(inv_date_sk#1)) - -(4) Scan parquet default.item -Output [1]: [i_item_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#5] - -(6) Filter [codegen id : 1] -Input [1]: [i_item_sk#5] -Condition : isnotnull(i_item_sk#5) - -(7) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#2] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_date_sk#1, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#5] -Input [5]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#5] - -(10) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(12) Filter [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#9] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#7] -Join condition: None - -(15) Project [codegen id : 4] -Output [5]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8] -Input [6]: [inv_date_sk#1, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8] - -(16) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_moy#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] - -(18) Filter [codegen id : 3] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 2001)) AND (d_moy#12 = 1)) AND isnotnull(d_date_sk#10)) - -(19) Project [codegen id : 3] -Output [2]: [d_date_sk#10, d_moy#12] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] - -(20) BroadcastExchange -Input [2]: [d_date_sk#10, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#1] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(22) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8, d_moy#12] -Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#10, d_moy#12] - -(23) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8, d_moy#12] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [5]: [n#14, avg#15, m2#16, sum#17, count#18] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] - -(24) Exchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, 5), true, [id=#24] - -(25) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stddev_samp(cast(inv_quantity_on_hand#4 as double))#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] - -(26) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stdev#27, mean#28] -Condition : ((CASE WHEN (mean#28 = 0.0) THEN 0.0 ELSE (stdev#27 / mean#28) END > 1.0) AND (CASE WHEN (mean#28 = 0.0) THEN null ELSE (stdev#27 / mean#28) END > 1.5)) - -(27) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, mean#28, CASE WHEN (mean#28 = 0.0) THEN null ELSE (stdev#27 / mean#28) END AS cov#29] -Input [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stdev#27, mean#28] - -(28) Scan parquet default.inventory -Output [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 8] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] - -(30) Filter [codegen id : 8] -Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4] -Condition : ((isnotnull(inv_item_sk#2) AND isnotnull(inv_warehouse_sk#3)) AND isnotnull(inv_date_sk#1)) - -(31) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#30] - -(32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#2] -Right keys [1]: [i_item_sk#30] -Join condition: None - -(33) Project [codegen id : 8] -Output [4]: [inv_date_sk#1, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#30] -Input [5]: [inv_date_sk#1, inv_item_sk#2, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#30] - -(34) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#31, w_warehouse_name#32] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#31] -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32] -Input [6]: [inv_date_sk#1, inv_warehouse_sk#3, inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32] - -(37) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_moy#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] - -(39) Filter [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] -Condition : ((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#33)) - -(40) Project [codegen id : 7] -Output [2]: [d_date_sk#33, d_moy#35] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] - -(41) BroadcastExchange -Input [2]: [d_date_sk#33, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] - -(42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#1] -Right keys [1]: [d_date_sk#33] -Join condition: None - -(43) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32, d_moy#35] -Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32, d_date_sk#33, d_moy#35] - -(44) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32, d_moy#35] -Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [5]: [n#37, avg#38, m2#39, sum#40, count#41] -Results [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] - -(45) Exchange -Input [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] -Arguments: hashpartitioning(w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, 5), true, [id=#47] - -(46) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] -Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#48, avg(cast(inv_quantity_on_hand#4 as bigint))#49] -Results [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#4 as double))#48 AS stdev#50, avg(cast(inv_quantity_on_hand#4 as bigint))#49 AS mean#51] - -(47) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stdev#50, mean#51] -Condition : (CASE WHEN (mean#51 = 0.0) THEN 0.0 ELSE (stdev#50 / mean#51) END > 1.0) - -(48) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, mean#51, CASE WHEN (mean#51 = 0.0) THEN null ELSE (stdev#50 / mean#51) END AS cov#52] -Input [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stdev#50, mean#51] - -(49) BroadcastExchange -Input [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, mean#51, cov#52] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#53] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [i_item_sk#5, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#30, w_warehouse_sk#31] -Join condition: None - -(51) Exchange -Input [10]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, mean#28, cov#29, w_warehouse_sk#31, i_item_sk#30, d_moy#35, mean#51, cov#52] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#5 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#28 ASC NULLS FIRST, cov#29 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#51 ASC NULLS FIRST, cov#52 ASC NULLS FIRST, 5), true, [id=#54] - -(52) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, mean#28, cov#29, w_warehouse_sk#31, i_item_sk#30, d_moy#35, mean#51, cov#52] -Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#5 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#28 ASC NULLS FIRST, cov#29 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#51 ASC NULLS FIRST, cov#52 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39b/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39b/simplified.txt deleted file mode 100644 index f4e23c837..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q39b/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -WholeStageCodegen (11) - Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] - InputAdapter - Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_date_sk,inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_date_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_date_sk,d_moy] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_date_sk,inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_date_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - InputAdapter - ReusedExchange [i_item_sk] #3 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - Project [d_date_sk,d_moy] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q4/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q4/explain.txt deleted file mode 100644 index 79a7abdcf..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q4/explain.txt +++ /dev/null @@ -1,606 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (107) -+- * Project (106) - +- * BroadcastHashJoin Inner BuildRight (105) - :- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- * Project (56) - : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : :- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * Filter (19) - : : : : : +- * HashAggregate (18) - : : : : : +- Exchange (17) - : : : : : +- * HashAggregate (16) - : : : : : +- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.customer (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet default.store_sales (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet default.date_dim (10) - : : : : +- BroadcastExchange (35) - : : : : +- * HashAggregate (34) - : : : : +- Exchange (33) - : : : : +- * HashAggregate (32) - : : : : +- * Project (31) - : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : :- * Project (25) - : : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : : :- * Filter (22) - : : : : : : +- * ColumnarToRow (21) - : : : : : : +- Scan parquet default.customer (20) - : : : : : +- ReusedExchange (23) - : : : : +- BroadcastExchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.date_dim (26) - : : : +- BroadcastExchange (54) - : : : +- * Project (53) - : : : +- * Filter (52) - : : : +- * HashAggregate (51) - : : : +- Exchange (50) - : : : +- * HashAggregate (49) - : : : +- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * Project (45) - : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet default.customer (37) - : : : : +- BroadcastExchange (43) - : : : : +- * Filter (42) - : : : : +- * ColumnarToRow (41) - : : : : +- Scan parquet default.catalog_sales (40) - : : : +- ReusedExchange (46) - : : +- BroadcastExchange (69) - : : +- * HashAggregate (68) - : : +- Exchange (67) - : : +- * HashAggregate (66) - : : +- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (62) - : : : +- * BroadcastHashJoin Inner BuildRight (61) - : : : :- * Filter (59) - : : : : +- * ColumnarToRow (58) - : : : : +- Scan parquet default.customer (57) - : : : +- ReusedExchange (60) - : : +- ReusedExchange (63) - : +- BroadcastExchange (89) - : +- * Project (88) - : +- * Filter (87) - : +- * HashAggregate (86) - : +- Exchange (85) - : +- * HashAggregate (84) - : +- * Project (83) - : +- * BroadcastHashJoin Inner BuildRight (82) - : :- * Project (80) - : : +- * BroadcastHashJoin Inner BuildRight (79) - : : :- * Filter (74) - : : : +- * ColumnarToRow (73) - : : : +- Scan parquet default.customer (72) - : : +- BroadcastExchange (78) - : : +- * Filter (77) - : : +- * ColumnarToRow (76) - : : +- Scan parquet default.web_sales (75) - : +- ReusedExchange (81) - +- BroadcastExchange (104) - +- * HashAggregate (103) - +- Exchange (102) - +- * HashAggregate (101) - +- * Project (100) - +- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (97) - : +- * BroadcastHashJoin Inner BuildRight (96) - : :- * Filter (94) - : : +- * ColumnarToRow (93) - : : +- Scan parquet default.customer (92) - : +- ReusedExchange (95) - +- ReusedExchange (98) - - -(1) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] - -(3) Filter [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(4) Scan parquet default.store_sales -Output [6]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [6]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14] - -(6) Filter [codegen id : 1] -Input [6]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14] -Condition : (isnotnull(ss_customer_sk#10) AND isnotnull(ss_sold_date_sk#9)) - -(7) BroadcastExchange -Input [6]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#15] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#10] -Join condition: None - -(9) Project [codegen id : 3] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14] - -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#16, d_year#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#16, d_year#17] - -(12) Filter [codegen id : 2] -Input [2]: [d_date_sk#16, d_year#17] -Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) - -(13) BroadcastExchange -Input [2]: [d_date_sk#16, d_year#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] - -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#16] -Join condition: None - -(15) Project [codegen id : 3] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14, d_year#17] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14, d_date_sk#16, d_year#17] - -(16) HashAggregate [codegen id : 3] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14, d_year#17] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#14 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#13 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#12 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#19, isEmpty#20] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#21, isEmpty#22] - -(17) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#21, isEmpty#22] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), true, [id=#23] - -(18) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#21, isEmpty#22] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#14 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#13 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#12 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#14 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#13 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#12 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#24] -Results [2]: [c_customer_id#2 AS customer_id#25, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#14 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#13 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#12 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#24 AS year_total#26] - -(19) Filter [codegen id : 24] -Input [2]: [customer_id#25, year_total#26] -Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.000000)) - -(20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] - -(22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(23) ReusedExchange [Reuses operator id: 7] -Output [6]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#10] -Join condition: None - -(25) Project [codegen id : 6] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14] - -(26) Scan parquet default.date_dim -Output [2]: [d_date_sk#16, d_year#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#16, d_year#17] - -(28) Filter [codegen id : 5] -Input [2]: [d_date_sk#16, d_year#17] -Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2002)) AND isnotnull(d_date_sk#16)) - -(29) BroadcastExchange -Input [2]: [d_date_sk#16, d_year#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#16] -Join condition: None - -(31) Project [codegen id : 6] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14, d_year#17] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_sold_date_sk#9, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14, d_date_sk#16, d_year#17] - -(32) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_ext_list_price#14, d_year#17] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#14 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#13 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#12 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#28, isEmpty#29] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#30, isEmpty#31] - -(33) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#30, isEmpty#31] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), true, [id=#32] - -(34) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#30, isEmpty#31] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#14 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#13 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#12 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#14 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#13 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#12 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#33] -Results [8]: [c_customer_id#2 AS customer_id#34, c_first_name#3 AS customer_first_name#35, c_last_name#4 AS customer_last_name#36, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#37, c_birth_country#6 AS customer_birth_country#38, c_login#7 AS customer_login#39, c_email_address#8 AS customer_email_address#40, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#14 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#13 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#11 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#12 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#33 AS year_total#41] - -(35) BroadcastExchange -Input [8]: [customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40, year_total#41] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] - -(36) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#34] -Join condition: None - -(37) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] - -(39) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(40) Scan parquet default.catalog_sales -Output [6]: [cs_sold_date_sk#43, cs_bill_customer_sk#44, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 8] -Input [6]: [cs_sold_date_sk#43, cs_bill_customer_sk#44, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48] - -(42) Filter [codegen id : 8] -Input [6]: [cs_sold_date_sk#43, cs_bill_customer_sk#44, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48] -Condition : (isnotnull(cs_bill_customer_sk#44) AND isnotnull(cs_sold_date_sk#43)) - -(43) BroadcastExchange -Input [6]: [cs_sold_date_sk#43, cs_bill_customer_sk#44, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#49] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_bill_customer_sk#44] -Join condition: None - -(45) Project [codegen id : 10] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_sold_date_sk#43, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_sold_date_sk#43, cs_bill_customer_sk#44, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48] - -(46) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#16, d_year#17] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#16] -Join condition: None - -(48) Project [codegen id : 10] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48, d_year#17] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_sold_date_sk#43, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48, d_date_sk#16, d_year#17] - -(49) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48, d_year#17] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#48 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#47 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#45 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#46 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#52, isEmpty#53] - -(50) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#52, isEmpty#53] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), true, [id=#54] - -(51) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#52, isEmpty#53] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#48 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#47 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#45 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#46 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#48 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#47 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#45 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#46 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#55] -Results [2]: [c_customer_id#2 AS customer_id#56, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#48 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#47 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#45 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#46 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#55 AS year_total#57] - -(52) Filter [codegen id : 11] -Input [2]: [customer_id#56, year_total#57] -Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.000000)) - -(53) Project [codegen id : 11] -Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] -Input [2]: [customer_id#56, year_total#57] - -(54) BroadcastExchange -Input [2]: [customer_id#58, year_total#59] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#60] - -(55) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#58] -Join condition: None - -(56) Project [codegen id : 24] -Output [11]: [customer_id#25, year_total#26, customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40, year_total#41, year_total#59] -Input [12]: [customer_id#25, year_total#26, customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40, year_total#41, customer_id#58, year_total#59] - -(57) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(58) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] - -(59) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(60) ReusedExchange [Reuses operator id: 43] -Output [6]: [cs_sold_date_sk#43, cs_bill_customer_sk#44, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48] - -(61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_bill_customer_sk#44] -Join condition: None - -(62) Project [codegen id : 14] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_sold_date_sk#43, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_sold_date_sk#43, cs_bill_customer_sk#44, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48] - -(63) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#16, d_year#17] - -(64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#16] -Join condition: None - -(65) Project [codegen id : 14] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48, d_year#17] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_sold_date_sk#43, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48, d_date_sk#16, d_year#17] - -(66) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#45, cs_ext_sales_price#46, cs_ext_wholesale_cost#47, cs_ext_list_price#48, d_year#17] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#48 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#47 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#45 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#46 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#61, isEmpty#62] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#63, isEmpty#64] - -(67) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#63, isEmpty#64] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), true, [id=#65] - -(68) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#63, isEmpty#64] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#48 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#47 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#45 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#46 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#48 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#47 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#45 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#46 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#66] -Results [2]: [c_customer_id#2 AS customer_id#67, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#48 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#47 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#45 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#46 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#66 AS year_total#68] - -(69) BroadcastExchange -Input [2]: [customer_id#67, year_total#68] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#69] - -(70) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#67] -Join condition: (CASE WHEN (year_total#59 > 0.000000) THEN CheckOverflow((promote_precision(year_total#68) / promote_precision(year_total#59)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#26 > 0.000000) THEN CheckOverflow((promote_precision(year_total#41) / promote_precision(year_total#26)), DecimalType(38,14), true) ELSE null END) - -(71) Project [codegen id : 24] -Output [10]: [customer_id#25, customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40, year_total#59, year_total#68] -Input [13]: [customer_id#25, year_total#26, customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40, year_total#41, year_total#59, customer_id#67, year_total#68] - -(72) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(73) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] - -(74) Filter [codegen id : 18] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(75) Scan parquet default.web_sales -Output [6]: [ws_sold_date_sk#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(76) ColumnarToRow [codegen id : 16] -Input [6]: [ws_sold_date_sk#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75] - -(77) Filter [codegen id : 16] -Input [6]: [ws_sold_date_sk#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75] -Condition : (isnotnull(ws_bill_customer_sk#71) AND isnotnull(ws_sold_date_sk#70)) - -(78) BroadcastExchange -Input [6]: [ws_sold_date_sk#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#76] - -(79) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#71] -Join condition: None - -(80) Project [codegen id : 18] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#70, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75] - -(81) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#16, d_year#17] - -(82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#70] -Right keys [1]: [d_date_sk#16] -Join condition: None - -(83) Project [codegen id : 18] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75, d_year#17] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#70, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75, d_date_sk#16, d_year#17] - -(84) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75, d_year#17] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#74 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#73 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#77, isEmpty#78] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#79, isEmpty#80] - -(85) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#79, isEmpty#80] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), true, [id=#81] - -(86) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#79, isEmpty#80] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#74 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#73 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#74 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#73 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#82] -Results [2]: [c_customer_id#2 AS customer_id#83, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#74 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#73 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#82 AS year_total#84] - -(87) Filter [codegen id : 19] -Input [2]: [customer_id#83, year_total#84] -Condition : (isnotnull(year_total#84) AND (year_total#84 > 0.000000)) - -(88) Project [codegen id : 19] -Output [2]: [customer_id#83 AS customer_id#85, year_total#84 AS year_total#86] -Input [2]: [customer_id#83, year_total#84] - -(89) BroadcastExchange -Input [2]: [customer_id#85, year_total#86] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#87] - -(90) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#85] -Join condition: None - -(91) Project [codegen id : 24] -Output [11]: [customer_id#25, customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40, year_total#59, year_total#68, year_total#86] -Input [12]: [customer_id#25, customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40, year_total#59, year_total#68, customer_id#85, year_total#86] - -(92) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(93) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] - -(94) Filter [codegen id : 22] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(95) ReusedExchange [Reuses operator id: 78] -Output [6]: [ws_sold_date_sk#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75] - -(96) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#71] -Join condition: None - -(97) Project [codegen id : 22] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#70, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75] - -(98) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#16, d_year#17] - -(99) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#70] -Right keys [1]: [d_date_sk#16] -Join condition: None - -(100) Project [codegen id : 22] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75, d_year#17] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_sold_date_sk#70, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75, d_date_sk#16, d_year#17] - -(101) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#72, ws_ext_sales_price#73, ws_ext_wholesale_cost#74, ws_ext_list_price#75, d_year#17] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#74 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#73 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#88, isEmpty#89] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#90, isEmpty#91] - -(102) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#90, isEmpty#91] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), true, [id=#92] - -(103) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#90, isEmpty#91] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#74 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#73 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#74 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#73 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#93] -Results [2]: [c_customer_id#2 AS customer_id#94, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#74 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#73 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#93 AS year_total#95] - -(104) BroadcastExchange -Input [2]: [customer_id#94, year_total#95] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] - -(105) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#94] -Join condition: (CASE WHEN (year_total#59 > 0.000000) THEN CheckOverflow((promote_precision(year_total#68) / promote_precision(year_total#59)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END) - -(106) Project [codegen id : 24] -Output [7]: [customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40] -Input [13]: [customer_id#25, customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40, year_total#59, year_total#68, year_total#86, customer_id#94, year_total#95] - -(107) TakeOrderedAndProject -Input [7]: [customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40] -Arguments: 100, [customer_id#34 ASC NULLS FIRST, customer_first_name#35 ASC NULLS FIRST, customer_last_name#36 ASC NULLS FIRST, customer_preferred_cust_flag#37 ASC NULLS FIRST, customer_birth_country#38 ASC NULLS FIRST, customer_login#39 ASC NULLS FIRST, customer_email_address#40 ASC NULLS FIRST], [customer_id#34, customer_first_name#35, customer_last_name#36, customer_preferred_cust_flag#37, customer_birth_country#38, customer_login#39, customer_email_address#40] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q4/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q4/simplified.txt deleted file mode 100644 index 017b34451..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q4/simplified.txt +++ /dev/null @@ -1,158 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - WholeStageCodegen (24) - Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_sold_date_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_sold_date_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - ReusedExchange [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - Project [customer_id,year_total] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #8 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_sold_date_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Filter [cs_bill_customer_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_sold_date_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - ReusedExchange [cs_sold_date_sk,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price] #9 - InputAdapter - ReusedExchange [d_date_sk,d_year] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (19) - Project [customer_id,year_total] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #13 - WholeStageCodegen (18) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_sold_date_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (16) - Filter [ws_bill_customer_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #16 - WholeStageCodegen (22) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_sold_date_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - ReusedExchange [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price] #14 - InputAdapter - ReusedExchange [d_date_sk,d_year] #6 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q40/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q40/explain.txt deleted file mode 100644 index 210469223..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q40/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin LeftOuter BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.catalog_returns (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.warehouse (10) - : +- BroadcastExchange (20) - : +- * Project (19) - : +- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet default.item (16) - +- BroadcastExchange (26) - +- * Filter (25) - +- * ColumnarToRow (24) - +- Scan parquet default.date_dim (23) - - -(1) Scan parquet default.catalog_sales -Output [5]: [cs_sold_date_sk#1, cs_warehouse_sk#2, cs_item_sk#3, cs_order_number#4, cs_sales_price#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [5]: [cs_sold_date_sk#1, cs_warehouse_sk#2, cs_item_sk#3, cs_order_number#4, cs_sales_price#5] - -(3) Filter [codegen id : 5] -Input [5]: [cs_sold_date_sk#1, cs_warehouse_sk#2, cs_item_sk#3, cs_order_number#4, cs_sales_price#5] -Condition : ((isnotnull(cs_warehouse_sk#2) AND isnotnull(cs_item_sk#3)) AND isnotnull(cs_sold_date_sk#1)) - -(4) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#6, cr_order_number#7, cr_refunded_cash#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [cr_item_sk#6, cr_order_number#7, cr_refunded_cash#8] - -(6) Filter [codegen id : 1] -Input [3]: [cr_item_sk#6, cr_order_number#7, cr_refunded_cash#8] -Condition : (isnotnull(cr_order_number#7) AND isnotnull(cr_item_sk#6)) - -(7) BroadcastExchange -Input [3]: [cr_item_sk#6, cr_order_number#7, cr_refunded_cash#8] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [id=#9] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [2]: [cs_order_number#4, cs_item_sk#3] -Right keys [2]: [cr_order_number#7, cr_item_sk#6] -Join condition: None - -(9) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#1, cs_warehouse_sk#2, cs_item_sk#3, cs_sales_price#5, cr_refunded_cash#8] -Input [8]: [cs_sold_date_sk#1, cs_warehouse_sk#2, cs_item_sk#3, cs_order_number#4, cs_sales_price#5, cr_item_sk#6, cr_order_number#7, cr_refunded_cash#8] - -(10) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#10, w_state#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#10, w_state#11] - -(12) Filter [codegen id : 2] -Input [2]: [w_warehouse_sk#10, w_state#11] -Condition : isnotnull(w_warehouse_sk#10) - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#10, w_state#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] - -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#10] -Join condition: None - -(15) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#1, cs_item_sk#3, cs_sales_price#5, cr_refunded_cash#8, w_state#11] -Input [7]: [cs_sold_date_sk#1, cs_warehouse_sk#2, cs_item_sk#3, cs_sales_price#5, cr_refunded_cash#8, w_warehouse_sk#10, w_state#11] - -(16) Scan parquet default.item -Output [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] - -(18) Filter [codegen id : 3] -Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] -Condition : (((isnotnull(i_current_price#15) AND (i_current_price#15 >= 0.99)) AND (i_current_price#15 <= 1.49)) AND isnotnull(i_item_sk#13)) - -(19) Project [codegen id : 3] -Output [2]: [i_item_sk#13, i_item_id#14] -Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] - -(20) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#13] -Join condition: None - -(22) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#1, cs_sales_price#5, cr_refunded_cash#8, w_state#11, i_item_id#14] -Input [7]: [cs_sold_date_sk#1, cs_item_sk#3, cs_sales_price#5, cr_refunded_cash#8, w_state#11, i_item_sk#13, i_item_id#14] - -(23) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_date#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#17, d_date#18] - -(25) Filter [codegen id : 4] -Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10997)) AND (d_date#18 <= 11057)) AND isnotnull(d_date_sk#17)) - -(26) BroadcastExchange -Input [2]: [d_date_sk#17, d_date#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] - -(27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#17] -Join condition: None - -(28) Project [codegen id : 5] -Output [5]: [cs_sales_price#5, cr_refunded_cash#8, w_state#11, i_item_id#14, d_date#18] -Input [7]: [cs_sold_date_sk#1, cs_sales_price#5, cr_refunded_cash#8, w_state#11, i_item_id#14, d_date_sk#17, d_date#18] - -(29) HashAggregate [codegen id : 5] -Input [5]: [cs_sales_price#5, cr_refunded_cash#8, w_state#11, i_item_id#14, d_date#18] -Keys [2]: [w_state#11, i_item_id#14] -Functions [2]: [partial_sum(CASE WHEN (d_date#18 < 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price#5 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#8 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#18 >= 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price#5 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#8 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END)] -Aggregate Attributes [4]: [sum#20, isEmpty#21, sum#22, isEmpty#23] -Results [6]: [w_state#11, i_item_id#14, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(30) Exchange -Input [6]: [w_state#11, i_item_id#14, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(w_state#11, i_item_id#14, 5), true, [id=#28] - -(31) HashAggregate [codegen id : 6] -Input [6]: [w_state#11, i_item_id#14, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [2]: [w_state#11, i_item_id#14] -Functions [2]: [sum(CASE WHEN (d_date#18 < 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price#5 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#8 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END), sum(CASE WHEN (d_date#18 >= 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price#5 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#8 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END)] -Aggregate Attributes [2]: [sum(CASE WHEN (d_date#18 < 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price#5 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#8 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END)#29, sum(CASE WHEN (d_date#18 >= 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price#5 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#8 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END)#30] -Results [4]: [w_state#11, i_item_id#14, sum(CASE WHEN (d_date#18 < 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price#5 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#8 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END)#29 AS sales_before#31, sum(CASE WHEN (d_date#18 >= 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price#5 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#8 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END)#30 AS sales_after#32] - -(32) TakeOrderedAndProject -Input [4]: [w_state#11, i_item_id#14, sales_before#31, sales_after#32] -Arguments: 100, [w_state#11 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#11, i_item_id#14, sales_before#31, sales_after#32] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q40/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q40/simplified.txt deleted file mode 100644 index b1e0f1f17..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q40/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - WholeStageCodegen (6) - HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END),sum(CASE WHEN (d_date >= 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_state,i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_sales_price,cr_refunded_cash,w_state,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_sold_date_sk,cs_item_sk,cs_sales_price,cr_refunded_cash,w_state] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Project [cs_sold_date_sk,cs_warehouse_sk,cs_item_sk,cs_sales_price,cr_refunded_cash] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - Filter [cs_warehouse_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [cr_order_number,cr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk,w_state] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id] - Filter [i_current_price,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q41/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q41/explain.txt deleted file mode 100644 index 13d73e61e..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q41/explain.txt +++ /dev/null @@ -1,120 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * BroadcastHashJoin Inner BuildRight (15) - :- * Project (4) - : +- * Filter (3) - : +- * ColumnarToRow (2) - : +- Scan parquet default.item (1) - +- BroadcastExchange (14) - +- * Project (13) - +- * Filter (12) - +- * HashAggregate (11) - +- Exchange (10) - +- * HashAggregate (9) - +- * Project (8) - +- * Filter (7) - +- * ColumnarToRow (6) - +- Scan parquet default.item (5) - - -(1) Scan parquet default.item -Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778), IsNotNull(i_manufact)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] - -(3) Filter [codegen id : 3] -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Condition : (((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) AND isnotnull(i_manufact#2)) - -(4) Project [codegen id : 3] -Output [2]: [i_manufact#2, i_product_name#3] -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] - -(5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] - -(7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) - -(8) Project [codegen id : 1] -Output [1]: [i_manufact#2] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] - -(9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#2] -Keys [1]: [i_manufact#2] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#8] -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] - -(11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#2, count#9] -Keys [1]: [i_manufact#2] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#11] -Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] - -(12) Filter [codegen id : 2] -Input [2]: [item_cnt#12, i_manufact#2#13] -Condition : (item_cnt#12 > 0) - -(13) Project [codegen id : 2] -Output [1]: [i_manufact#2#13] -Input [2]: [item_cnt#12, i_manufact#2#13] - -(14) BroadcastExchange -Input [1]: [i_manufact#2#13] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#2#13] -Join condition: None - -(16) Project [codegen id : 3] -Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] - -(17) HashAggregate [codegen id : 3] -Input [1]: [i_product_name#3] -Keys [1]: [i_product_name#3] -Functions: [] -Aggregate Attributes: [] -Results [1]: [i_product_name#3] - -(18) Exchange -Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] - -(19) HashAggregate [codegen id : 4] -Input [1]: [i_product_name#3] -Keys [1]: [i_product_name#3] -Functions: [] -Aggregate Attributes: [] -Results [1]: [i_product_name#3] - -(20) TakeOrderedAndProject -Input [1]: [i_product_name#3] -Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q41/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q41/simplified.txt deleted file mode 100644 index 2d14d75ca..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q41/simplified.txt +++ /dev/null @@ -1,29 +0,0 @@ -TakeOrderedAndProject [i_product_name] - WholeStageCodegen (4) - HashAggregate [i_product_name] - InputAdapter - Exchange [i_product_name] #1 - WholeStageCodegen (3) - HashAggregate [i_product_name] - Project [i_product_name] - BroadcastHashJoin [i_manufact,i_manufact] - Project [i_manufact,i_product_name] - Filter [i_manufact_id,i_manufact] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_manufact_id,i_manufact,i_product_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [i_manufact] - Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] - InputAdapter - Exchange [i_manufact] #3 - WholeStageCodegen (1) - HashAggregate [i_manufact] [count,count] - Project [i_manufact] - Filter [i_category,i_color,i_units,i_size,i_manufact] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q42/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q42/explain.txt deleted file mode 100644 index f7732f3c8..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q42/explain.txt +++ /dev/null @@ -1,122 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.date_dim (1) - : +- BroadcastExchange (8) - : +- * Filter (7) - : +- * ColumnarToRow (6) - : +- Scan parquet default.store_sales (5) - +- BroadcastExchange (15) - +- * Project (14) - +- * Filter (13) - +- * ColumnarToRow (12) - +- Scan parquet default.item (11) - - -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] - -(3) Filter [codegen id : 3] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(4) Project [codegen id : 3] -Output [2]: [d_date_sk#1, d_year#2] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] - -(5) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] - -(7) Filter [codegen id : 1] -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Condition : (isnotnull(ss_sold_date_sk#4) AND isnotnull(ss_item_sk#5)) - -(8) BroadcastExchange -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#5, ss_ext_sales_price#6] -Input [5]: [d_date_sk#1, d_year#2, ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] - -(11) Scan parquet default.item -Output [4]: [i_item_sk#8, i_category_id#9, i_category#10, i_manager_id#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [4]: [i_item_sk#8, i_category_id#9, i_category#10, i_manager_id#11] - -(13) Filter [codegen id : 2] -Input [4]: [i_item_sk#8, i_category_id#9, i_category#10, i_manager_id#11] -Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 1)) AND isnotnull(i_item_sk#8)) - -(14) Project [codegen id : 2] -Output [3]: [i_item_sk#8, i_category_id#9, i_category#10] -Input [4]: [i_item_sk#8, i_category_id#9, i_category#10, i_manager_id#11] - -(15) BroadcastExchange -Input [3]: [i_item_sk#8, i_category_id#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#8] -Join condition: None - -(17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#6, i_category_id#9, i_category#10] -Input [6]: [d_year#2, ss_item_sk#5, ss_ext_sales_price#6, i_item_sk#8, i_category_id#9, i_category#10] - -(18) HashAggregate [codegen id : 3] -Input [4]: [d_year#2, ss_ext_sales_price#6, i_category_id#9, i_category#10] -Keys [3]: [d_year#2, i_category_id#9, i_category#10] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum#13] -Results [4]: [d_year#2, i_category_id#9, i_category#10, sum#14] - -(19) Exchange -Input [4]: [d_year#2, i_category_id#9, i_category#10, sum#14] -Arguments: hashpartitioning(d_year#2, i_category_id#9, i_category#10, 5), true, [id=#15] - -(20) HashAggregate [codegen id : 4] -Input [4]: [d_year#2, i_category_id#9, i_category#10, sum#14] -Keys [3]: [d_year#2, i_category_id#9, i_category#10] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#16] -Results [4]: [d_year#2, i_category_id#9, i_category#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#16,17,2) AS sum(ss_ext_sales_price)#17] - -(21) TakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17] -Arguments: 100, [sum(ss_ext_sales_price)#17 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q42/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q42/simplified.txt deleted file mode 100644 index d9bb6de20..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q42/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] - WholeStageCodegen (4) - HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] - InputAdapter - Exchange [d_year,i_category_id,i_category] #1 - WholeStageCodegen (3) - HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_category_id,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [d_date_sk,d_year] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [i_item_sk,i_category_id,i_category] - Filter [i_manager_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q43/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q43/explain.txt deleted file mode 100644 index 8f3ef7fee..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q43/explain.txt +++ /dev/null @@ -1,122 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.date_dim (1) - : +- BroadcastExchange (8) - : +- * Filter (7) - : +- * ColumnarToRow (6) - : +- Scan parquet default.store_sales (5) - +- BroadcastExchange (15) - +- * Project (14) - +- * Filter (13) - +- * ColumnarToRow (12) - +- Scan parquet default.store (11) - - -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] - -(3) Filter [codegen id : 3] -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(4) Project [codegen id : 3] -Output [2]: [d_date_sk#1, d_day_name#3] -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] - -(5) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#4, ss_store_sk#5, ss_sales_price#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_sold_date_sk#4, ss_store_sk#5, ss_sales_price#6] - -(7) Filter [codegen id : 1] -Input [3]: [ss_sold_date_sk#4, ss_store_sk#5, ss_sales_price#6] -Condition : (isnotnull(ss_sold_date_sk#4) AND isnotnull(ss_store_sk#5)) - -(8) BroadcastExchange -Input [3]: [ss_sold_date_sk#4, ss_store_sk#5, ss_sales_price#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [d_day_name#3, ss_store_sk#5, ss_sales_price#6] -Input [5]: [d_date_sk#1, d_day_name#3, ss_sold_date_sk#4, ss_store_sk#5, ss_sales_price#6] - -(11) Scan parquet default.store -Output [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] - -(13) Filter [codegen id : 2] -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Condition : ((isnotnull(s_gmt_offset#11) AND (s_gmt_offset#11 = -5.00)) AND isnotnull(s_store_sk#8)) - -(14) Project [codegen id : 2] -Output [3]: [s_store_sk#8, s_store_id#9, s_store_name#10] -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] - -(15) BroadcastExchange -Input [3]: [s_store_sk#8, s_store_id#9, s_store_name#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#8] -Join condition: None - -(17) Project [codegen id : 3] -Output [4]: [d_day_name#3, ss_sales_price#6, s_store_id#9, s_store_name#10] -Input [6]: [d_day_name#3, ss_store_sk#5, ss_sales_price#6, s_store_sk#8, s_store_id#9, s_store_name#10] - -(18) HashAggregate [codegen id : 3] -Input [4]: [d_day_name#3, ss_sales_price#6, s_store_id#9, s_store_name#10] -Keys [2]: [s_store_name#10, s_store_id#9] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday) THEN ss_sales_price#6 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday) THEN ss_sales_price#6 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday) THEN ss_sales_price#6 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#6 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday) THEN ss_sales_price#6 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday) THEN ss_sales_price#6 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday) THEN ss_sales_price#6 ELSE null END))] -Aggregate Attributes [7]: [sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Results [9]: [s_store_name#10, s_store_id#9, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] - -(19) Exchange -Input [9]: [s_store_name#10, s_store_id#9, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] -Arguments: hashpartitioning(s_store_name#10, s_store_id#9, 5), true, [id=#27] - -(20) HashAggregate [codegen id : 4] -Input [9]: [s_store_name#10, s_store_id#9, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] -Keys [2]: [s_store_name#10, s_store_id#9] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday) THEN ss_sales_price#6 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday) THEN ss_sales_price#6 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday) THEN ss_sales_price#6 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#6 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday) THEN ss_sales_price#6 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday) THEN ss_sales_price#6 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday) THEN ss_sales_price#6 ELSE null END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday) THEN ss_sales_price#6 ELSE null END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday) THEN ss_sales_price#6 ELSE null END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday) THEN ss_sales_price#6 ELSE null END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#6 ELSE null END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday) THEN ss_sales_price#6 ELSE null END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday) THEN ss_sales_price#6 ELSE null END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday) THEN ss_sales_price#6 ELSE null END))#34] -Results [9]: [s_store_name#10, s_store_id#9, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday) THEN ss_sales_price#6 ELSE null END))#28,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday) THEN ss_sales_price#6 ELSE null END))#29,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday) THEN ss_sales_price#6 ELSE null END))#30,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#6 ELSE null END))#31,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday) THEN ss_sales_price#6 ELSE null END))#32,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday) THEN ss_sales_price#6 ELSE null END))#33,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday) THEN ss_sales_price#6 ELSE null END))#34,17,2) AS sat_sales#41] - -(21) TakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST, s_store_id#9 ASC NULLS FIRST, sun_sales#35 ASC NULLS FIRST, mon_sales#36 ASC NULLS FIRST, tue_sales#37 ASC NULLS FIRST, wed_sales#38 ASC NULLS FIRST, thu_sales#39 ASC NULLS FIRST, fri_sales#40 ASC NULLS FIRST, sat_sales#41 ASC NULLS FIRST], [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q43/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q43/simplified.txt deleted file mode 100644 index 1694f3aab..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q43/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - WholeStageCodegen (4) - HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - InputAdapter - Exchange [s_store_name,s_store_id] #1 - WholeStageCodegen (3) - HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,ss_sales_price,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [d_day_name,ss_store_sk,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [d_date_sk,d_day_name] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_day_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [s_store_sk,s_store_id,s_store_name] - Filter [s_gmt_offset,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q44/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q44/explain.txt deleted file mode 100644 index 096bd45f0..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q44/explain.txt +++ /dev/null @@ -1,248 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (36) -+- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (14) - : : : +- * Filter (13) - : : : +- Window (12) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * Project (9) - : : : +- * Filter (8) - : : : +- * HashAggregate (7) - : : : +- Exchange (6) - : : : +- * HashAggregate (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * Filter (22) - : : +- Window (21) - : : +- * Sort (20) - : : +- Exchange (19) - : : +- * Project (18) - : : +- * Filter (17) - : : +- * HashAggregate (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.item (27) - +- ReusedExchange (33) - - -(1) Scan parquet default.store_sales -Output [3]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3] - -(3) Filter [codegen id : 1] -Input [3]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3] -Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) - -(4) Project [codegen id : 1] -Output [2]: [ss_item_sk#1, ss_net_profit#3] -Input [3]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3] - -(5) HashAggregate [codegen id : 1] -Input [2]: [ss_item_sk#1, ss_net_profit#3] -Keys [1]: [ss_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#4, count#5] -Results [3]: [ss_item_sk#1, sum#6, count#7] - -(6) Exchange -Input [3]: [ss_item_sk#1, sum#6, count#7] -Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#8] - -(7) HashAggregate [codegen id : 2] -Input [3]: [ss_item_sk#1, sum#6, count#7] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#9] -Results [3]: [ss_item_sk#1 AS item_sk#10, cast((avg(UnscaledValue(ss_net_profit#3))#9 / 100.0) as decimal(11,6)) AS rank_col#11, cast((avg(UnscaledValue(ss_net_profit#3))#9 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#12] - -(8) Filter [codegen id : 2] -Input [3]: [item_sk#10, rank_col#11, avg(ss_net_profit#3)#12] -Condition : (isnotnull(avg(ss_net_profit#3)#12) AND (cast(avg(ss_net_profit#3)#12 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(Subquery scalar-subquery#13, [id=#14])), DecimalType(13,7), true))) - -(9) Project [codegen id : 2] -Output [2]: [item_sk#10, rank_col#11] -Input [3]: [item_sk#10, rank_col#11, avg(ss_net_profit#3)#12] - -(10) Exchange -Input [2]: [item_sk#10, rank_col#11] -Arguments: SinglePartition, true, [id=#15] - -(11) Sort [codegen id : 3] -Input [2]: [item_sk#10, rank_col#11] -Arguments: [rank_col#11 ASC NULLS FIRST], false, 0 - -(12) Window -Input [2]: [item_sk#10, rank_col#11] -Arguments: [rank(rank_col#11) windowspecdefinition(rank_col#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#16], [rank_col#11 ASC NULLS FIRST] - -(13) Filter [codegen id : 10] -Input [3]: [item_sk#10, rank_col#11, rnk#16] -Condition : ((isnotnull(rnk#16) AND (rnk#16 < 11)) AND isnotnull(item_sk#10)) - -(14) Project [codegen id : 10] -Output [2]: [item_sk#10, rnk#16] -Input [3]: [item_sk#10, rank_col#11, rnk#16] - -(15) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#1, sum#17, count#18] - -(16) HashAggregate [codegen id : 5] -Input [3]: [ss_item_sk#1, sum#17, count#18] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#19] -Results [3]: [ss_item_sk#1 AS item_sk#20, cast((avg(UnscaledValue(ss_net_profit#3))#19 / 100.0) as decimal(11,6)) AS rank_col#21, cast((avg(UnscaledValue(ss_net_profit#3))#19 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#22] - -(17) Filter [codegen id : 5] -Input [3]: [item_sk#20, rank_col#21, avg(ss_net_profit#3)#22] -Condition : (isnotnull(avg(ss_net_profit#3)#22) AND (cast(avg(ss_net_profit#3)#22 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#13, [id=#14])), DecimalType(13,7), true))) - -(18) Project [codegen id : 5] -Output [2]: [item_sk#20, rank_col#21] -Input [3]: [item_sk#20, rank_col#21, avg(ss_net_profit#3)#22] - -(19) Exchange -Input [2]: [item_sk#20, rank_col#21] -Arguments: SinglePartition, true, [id=#23] - -(20) Sort [codegen id : 6] -Input [2]: [item_sk#20, rank_col#21] -Arguments: [rank_col#21 DESC NULLS LAST], false, 0 - -(21) Window -Input [2]: [item_sk#20, rank_col#21] -Arguments: [rank(rank_col#21) windowspecdefinition(rank_col#21 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#24], [rank_col#21 DESC NULLS LAST] - -(22) Filter [codegen id : 7] -Input [3]: [item_sk#20, rank_col#21, rnk#24] -Condition : ((isnotnull(rnk#24) AND (rnk#24 < 11)) AND isnotnull(item_sk#20)) - -(23) Project [codegen id : 7] -Output [2]: [item_sk#20, rnk#24] -Input [3]: [item_sk#20, rank_col#21, rnk#24] - -(24) BroadcastExchange -Input [2]: [item_sk#20, rnk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#25] - -(25) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [rnk#16] -Right keys [1]: [rnk#24] -Join condition: None - -(26) Project [codegen id : 10] -Output [3]: [item_sk#10, rnk#16, item_sk#20] -Input [4]: [item_sk#10, rnk#16, item_sk#20, rnk#24] - -(27) Scan parquet default.item -Output [2]: [i_item_sk#26, i_product_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(28) ColumnarToRow [codegen id : 8] -Input [2]: [i_item_sk#26, i_product_name#27] - -(29) Filter [codegen id : 8] -Input [2]: [i_item_sk#26, i_product_name#27] -Condition : isnotnull(i_item_sk#26) - -(30) BroadcastExchange -Input [2]: [i_item_sk#26, i_product_name#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] - -(31) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#10] -Right keys [1]: [i_item_sk#26] -Join condition: None - -(32) Project [codegen id : 10] -Output [3]: [rnk#16, item_sk#20, i_product_name#27] -Input [5]: [item_sk#10, rnk#16, item_sk#20, i_item_sk#26, i_product_name#27] - -(33) ReusedExchange [Reuses operator id: 30] -Output [2]: [i_item_sk#29, i_product_name#30] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#20] -Right keys [1]: [i_item_sk#29] -Join condition: None - -(35) Project [codegen id : 10] -Output [3]: [rnk#16, i_product_name#27 AS best_performing#31, i_product_name#30 AS worst_performing#32] -Input [5]: [rnk#16, item_sk#20, i_product_name#27, i_item_sk#29, i_product_name#30] - -(36) TakeOrderedAndProject -Input [3]: [rnk#16, best_performing#31, worst_performing#32] -Arguments: 100, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#31, worst_performing#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#13, [id=#14] -* HashAggregate (43) -+- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet default.store_sales (37) - - -(37) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 1] -Input [3]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3] - -(39) Filter [codegen id : 1] -Input [3]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3] -Condition : ((isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) AND isnull(ss_addr_sk#33)) - -(40) Project [codegen id : 1] -Output [2]: [ss_store_sk#2, ss_net_profit#3] -Input [3]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3] - -(41) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#2, ss_net_profit#3] -Keys [1]: [ss_store_sk#2] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#34, count#35] -Results [3]: [ss_store_sk#2, sum#36, count#37] - -(42) Exchange -Input [3]: [ss_store_sk#2, sum#36, count#37] -Arguments: hashpartitioning(ss_store_sk#2, 5), true, [id=#38] - -(43) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#2, sum#36, count#37] -Keys [1]: [ss_store_sk#2] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#39] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#3))#39 / 100.0) as decimal(11,6)) AS rank_col#40] - -Subquery:2 Hosting operator id = 17 Hosting Expression = ReusedSubquery Subquery scalar-subquery#13, [id=#14] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q44/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q44/simplified.txt deleted file mode 100644 index f2106ad7a..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q44/simplified.txt +++ /dev/null @@ -1,68 +0,0 @@ -TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (10) - Project [rnk,i_product_name,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [rnk,item_sk,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [item_sk,rnk,item_sk] - BroadcastHashJoin [rnk,rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (3) - Sort [rank_col] - InputAdapter - Exchange #1 - WholeStageCodegen (2) - Project [item_sk,rank_col] - Filter [avg(ss_net_profit)] - Subquery #1 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count] - InputAdapter - Exchange [ss_store_sk] #3 - WholeStageCodegen (1) - HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count] - Project [ss_store_sk,ss_net_profit] - Filter [ss_store_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit] - HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,avg(ss_net_profit),sum,count] - InputAdapter - Exchange [ss_item_sk] #2 - WholeStageCodegen (1) - HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count] - Project [ss_item_sk,ss_net_profit] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (6) - Sort [rank_col] - InputAdapter - Exchange #5 - WholeStageCodegen (5) - Project [item_sk,rank_col] - Filter [avg(ss_net_profit)] - ReusedSubquery [rank_col] #1 - HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,avg(ss_net_profit),sum,count] - InputAdapter - ReusedExchange [ss_item_sk,sum,count] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_product_name] - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #6 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q45/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q45/explain.txt deleted file mode 100644 index f556e9b0d..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q45/explain.txt +++ /dev/null @@ -1,226 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * Filter (35) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (34) - :- * Project (28) - : +- * BroadcastHashJoin Inner BuildRight (27) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.customer (4) - : : : +- BroadcastExchange (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.customer_address (10) - : : +- BroadcastExchange (20) - : : +- * Project (19) - : : +- * Filter (18) - : : +- * ColumnarToRow (17) - : : +- Scan parquet default.date_dim (16) - : +- BroadcastExchange (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.item (23) - +- BroadcastExchange (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.item (29) - - -(1) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#2, ws_item_sk#3, ws_bill_customer_sk#4, ws_sales_price#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [4]: [ws_sold_date_sk#2, ws_item_sk#3, ws_bill_customer_sk#4, ws_sales_price#5] - -(3) Filter [codegen id : 6] -Input [4]: [ws_sold_date_sk#2, ws_item_sk#3, ws_bill_customer_sk#4, ws_sales_price#5] -Condition : ((isnotnull(ws_bill_customer_sk#4) AND isnotnull(ws_sold_date_sk#2)) AND isnotnull(ws_item_sk#3)) - -(4) Scan parquet default.customer -Output [2]: [c_customer_sk#6, c_current_addr_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#6, c_current_addr_sk#7] - -(6) Filter [codegen id : 1] -Input [2]: [c_customer_sk#6, c_current_addr_sk#7] -Condition : (isnotnull(c_customer_sk#6) AND isnotnull(c_current_addr_sk#7)) - -(7) BroadcastExchange -Input [2]: [c_customer_sk#6, c_current_addr_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_bill_customer_sk#4] -Right keys [1]: [c_customer_sk#6] -Join condition: None - -(9) Project [codegen id : 6] -Output [4]: [ws_sold_date_sk#2, ws_item_sk#3, ws_sales_price#5, c_current_addr_sk#7] -Input [6]: [ws_sold_date_sk#2, ws_item_sk#3, ws_bill_customer_sk#4, ws_sales_price#5, c_customer_sk#6, c_current_addr_sk#7] - -(10) Scan parquet default.customer_address -Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] - -(12) Filter [codegen id : 2] -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) - -(13) BroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] - -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_addr_sk#7] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(15) Project [codegen id : 6] -Output [5]: [ws_sold_date_sk#2, ws_item_sk#3, ws_sales_price#5, ca_city#10, ca_zip#11] -Input [7]: [ws_sold_date_sk#2, ws_item_sk#3, ws_sales_price#5, c_current_addr_sk#7, ca_address_sk#9, ca_city#10, ca_zip#11] - -(16) Scan parquet default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] - -(18) Filter [codegen id : 3] -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(19) Project [codegen id : 3] -Output [1]: [d_date_sk#13] -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] - -(20) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#2] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(22) Project [codegen id : 6] -Output [4]: [ws_item_sk#3, ws_sales_price#5, ca_city#10, ca_zip#11] -Input [6]: [ws_sold_date_sk#2, ws_item_sk#3, ws_sales_price#5, ca_city#10, ca_zip#11, d_date_sk#13] - -(23) Scan parquet default.item -Output [2]: [i_item_sk#17, i_item_id#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#17, i_item_id#18] - -(25) Filter [codegen id : 4] -Input [2]: [i_item_sk#17, i_item_id#18] -Condition : isnotnull(i_item_sk#17) - -(26) BroadcastExchange -Input [2]: [i_item_sk#17, i_item_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#3] -Right keys [1]: [i_item_sk#17] -Join condition: None - -(28) Project [codegen id : 6] -Output [4]: [ws_sales_price#5, ca_city#10, ca_zip#11, i_item_id#18] -Input [6]: [ws_item_sk#3, ws_sales_price#5, ca_city#10, ca_zip#11, i_item_sk#17, i_item_id#18] - -(29) Scan parquet default.item -Output [2]: [i_item_sk#17, i_item_id#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#17, i_item_id#18] - -(31) Filter [codegen id : 5] -Input [2]: [i_item_sk#17, i_item_id#18] -Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) - -(32) Project [codegen id : 5] -Output [1]: [i_item_id#18 AS i_item_id#18#20] -Input [2]: [i_item_sk#17, i_item_id#18] - -(33) BroadcastExchange -Input [1]: [i_item_id#18#20] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#21] - -(34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_id#18] -Right keys [1]: [i_item_id#18#20] -Join condition: None - -(35) Filter [codegen id : 6] -Input [5]: [ws_sales_price#5, ca_city#10, ca_zip#11, i_item_id#18, exists#1] -Condition : (substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) - -(36) Project [codegen id : 6] -Output [3]: [ws_sales_price#5, ca_city#10, ca_zip#11] -Input [5]: [ws_sales_price#5, ca_city#10, ca_zip#11, i_item_id#18, exists#1] - -(37) HashAggregate [codegen id : 6] -Input [3]: [ws_sales_price#5, ca_city#10, ca_zip#11] -Keys [2]: [ca_zip#11, ca_city#10] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#5))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [ca_zip#11, ca_city#10, sum#23] - -(38) Exchange -Input [3]: [ca_zip#11, ca_city#10, sum#23] -Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), true, [id=#24] - -(39) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#11, ca_city#10, sum#23] -Keys [2]: [ca_zip#11, ca_city#10] -Functions [1]: [sum(UnscaledValue(ws_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#5))#25] -Results [3]: [ca_zip#11, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#5))#25,17,2) AS sum(ws_sales_price)#26] - -(40) TakeOrderedAndProject -Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#26] -Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#26] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q45/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q45/simplified.txt deleted file mode 100644 index 7cc474fb5..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q45/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (7) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - InputAdapter - Exchange [ca_zip,ca_city] #1 - WholeStageCodegen (6) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - Project [ws_sales_price,ca_city,ca_zip,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_item_sk,ws_sales_price,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ws_sold_date_sk,ws_item_sk,ws_sales_price,c_current_addr_sk] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Filter [ws_bill_customer_sk,ws_sold_date_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_sales_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city,ca_zip] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q46/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q46/explain.txt deleted file mode 100644 index a6a3c3c46..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q46/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * HashAggregate (33) - : : +- Exchange (32) - : : +- * HashAggregate (31) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet default.store_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * Project (7) - : : : : : +- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.date_dim (4) - : : : : +- BroadcastExchange (15) - : : : : +- * Project (14) - : : : : +- * Filter (13) - : : : : +- * ColumnarToRow (12) - : : : : +- Scan parquet default.store (11) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet default.household_demographics (18) - : : +- BroadcastExchange (28) - : : +- * Filter (27) - : : +- * ColumnarToRow (26) - : : +- Scan parquet default.customer_address (25) - : +- BroadcastExchange (37) - : +- * Filter (36) - : +- * ColumnarToRow (35) - : +- Scan parquet default.customer (34) - +- ReusedExchange (40) - - -(1) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8] -Condition : ((((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#5)) AND isnotnull(ss_hdemo_sk#3)) AND isnotnull(ss_addr_sk#4)) AND isnotnull(ss_customer_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_dow#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [6,0]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#9, d_year#10, d_dow#11] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#9, d_year#10, d_dow#11] -Condition : ((d_dow#11 IN (6,0) AND d_year#10 IN (1999,2000,2001)) AND isnotnull(d_date_sk#9)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#9] -Input [3]: [d_date_sk#9, d_year#10, d_dow#11] - -(8) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#9] -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8] -Input [9]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, d_date_sk#9] - -(11) Scan parquet default.store -Output [2]: [s_store_sk#13, s_city#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#13, s_city#14] - -(13) Filter [codegen id : 2] -Input [2]: [s_store_sk#13, s_city#14] -Condition : (s_city#14 IN (Fairview,Midway) AND isnotnull(s_store_sk#13)) - -(14) Project [codegen id : 2] -Output [1]: [s_store_sk#13] -Input [2]: [s_store_sk#13, s_city#14] - -(15) BroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#13] -Join condition: None - -(17) Project [codegen id : 5] -Output [6]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8] -Input [8]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, s_store_sk#13] - -(18) Scan parquet default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] - -(20) Filter [codegen id : 3] -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 4) OR (hd_vehicle_count#18 = 3)) AND isnotnull(hd_demo_sk#16)) - -(21) Project [codegen id : 3] -Output [1]: [hd_demo_sk#16] -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] - -(22) BroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#16] -Join condition: None - -(24) Project [codegen id : 5] -Output [5]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8] -Input [7]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, hd_demo_sk#16] - -(25) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_city#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#20, ca_city#21] - -(27) Filter [codegen id : 4] -Input [2]: [ca_address_sk#20, ca_city#21] -Condition : (isnotnull(ca_address_sk#20) AND isnotnull(ca_city#21)) - -(28) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_city#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] - -(29) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#4] -Right keys [1]: [ca_address_sk#20] -Join condition: None - -(30) Project [codegen id : 5] -Output [6]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, ca_city#21] -Input [7]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, ca_address_sk#20, ca_city#21] - -(31) HashAggregate [codegen id : 5] -Input [6]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, ca_city#21] -Keys [4]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#21] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#7)), partial_sum(UnscaledValue(ss_net_profit#8))] -Aggregate Attributes [2]: [sum#23, sum#24] -Results [6]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#21, sum#25, sum#26] - -(32) Exchange -Input [6]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#21, sum#25, sum#26] -Arguments: hashpartitioning(ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#21, 5), true, [id=#27] - -(33) HashAggregate [codegen id : 8] -Input [6]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#21, sum#25, sum#26] -Keys [4]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#21] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#7)), sum(UnscaledValue(ss_net_profit#8))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#7))#28, sum(UnscaledValue(ss_net_profit#8))#29] -Results [5]: [ss_ticket_number#6, ss_customer_sk#2, ca_city#21 AS bought_city#30, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#7))#28,17,2) AS amt#31, MakeDecimal(sum(UnscaledValue(ss_net_profit#8))#29,17,2) AS profit#32] - -(34) Scan parquet default.customer -Output [4]: [c_customer_sk#33, c_current_addr_sk#34, c_first_name#35, c_last_name#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#33, c_current_addr_sk#34, c_first_name#35, c_last_name#36] - -(36) Filter [codegen id : 6] -Input [4]: [c_customer_sk#33, c_current_addr_sk#34, c_first_name#35, c_last_name#36] -Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#34)) - -(37) BroadcastExchange -Input [4]: [c_customer_sk#33, c_current_addr_sk#34, c_first_name#35, c_last_name#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] - -(38) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#33] -Join condition: None - -(39) Project [codegen id : 8] -Output [7]: [ss_ticket_number#6, bought_city#30, amt#31, profit#32, c_current_addr_sk#34, c_first_name#35, c_last_name#36] -Input [9]: [ss_ticket_number#6, ss_customer_sk#2, bought_city#30, amt#31, profit#32, c_customer_sk#33, c_current_addr_sk#34, c_first_name#35, c_last_name#36] - -(40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#20, ca_city#21] - -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#20] -Join condition: NOT (ca_city#21 = bought_city#30) - -(42) Project [codegen id : 8] -Output [7]: [c_last_name#36, c_first_name#35, ca_city#21, bought_city#30, ss_ticket_number#6, amt#31, profit#32] -Input [9]: [ss_ticket_number#6, bought_city#30, amt#31, profit#32, c_current_addr_sk#34, c_first_name#35, c_last_name#36, ca_address_sk#20, ca_city#21] - -(43) TakeOrderedAndProject -Input [7]: [c_last_name#36, c_first_name#35, ca_city#21, bought_city#30, ss_ticket_number#6, amt#31, profit#32] -Arguments: 100, [c_last_name#36 ASC NULLS FIRST, c_first_name#35 ASC NULLS FIRST, ca_city#21 ASC NULLS FIRST, bought_city#30 ASC NULLS FIRST, ss_ticket_number#6 ASC NULLS FIRST], [c_last_name#36, c_first_name#35, ca_city#21, bought_city#30, ss_ticket_number#6, amt#31, profit#32] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q46/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q46/simplified.txt deleted file mode 100644 index abdc7a3ba..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q46/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (8) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - InputAdapter - Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_dow,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [s_store_sk] - Filter [s_city,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [hd_demo_sk] - Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [ca_address_sk,ca_city] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q47/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q47/explain.txt deleted file mode 100644 index a84ba3a03..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q47/explain.txt +++ /dev/null @@ -1,278 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Filter (32) - : : +- Window (31) - : : +- * Sort (30) - : : +- Exchange (29) - : : +- * Project (28) - : : +- Window (27) - : : +- * Sort (26) - : : +- Exchange (25) - : : +- * HashAggregate (24) - : : +- Exchange (23) - : : +- * HashAggregate (22) - : : +- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_sales (4) - : : : +- BroadcastExchange (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (19) - : : +- * Filter (18) - : : +- * ColumnarToRow (17) - : : +- Scan parquet default.store (16) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- * Filter (38) - : +- Window (37) - : +- * Sort (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- ReusedExchange (33) - +- BroadcastExchange (48) - +- * Project (47) - +- * Filter (46) - +- Window (45) - +- * Sort (44) - +- ReusedExchange (43) - - -(1) Scan parquet default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] - -(3) Filter [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) - -(4) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [ss_sold_date_sk#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7] - -(6) Filter [codegen id : 1] -Input [4]: [ss_sold_date_sk#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7] -Condition : ((isnotnull(ss_item_sk#5) AND isnotnull(ss_sold_date_sk#4)) AND isnotnull(ss_store_sk#6)) - -(7) BroadcastExchange -Input [4]: [ss_sold_date_sk#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#8] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#5] -Join condition: None - -(9) Project [codegen id : 4] -Output [5]: [i_brand#2, i_category#3, ss_sold_date_sk#4, ss_store_sk#6, ss_sales_price#7] -Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_sold_date_sk#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7] - -(10) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] - -(12) Filter [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) - -(13) BroadcastExchange -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join condition: None - -(15) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, ss_store_sk#6, ss_sales_price#7, d_year#10, d_moy#11] -Input [8]: [i_brand#2, i_category#3, ss_sold_date_sk#4, ss_store_sk#6, ss_sales_price#7, d_date_sk#9, d_year#10, d_moy#11] - -(16) Scan parquet default.store -Output [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] - -(18) Filter [codegen id : 3] -Input [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] -Condition : ((isnotnull(s_store_sk#13) AND isnotnull(s_store_name#14)) AND isnotnull(s_company_name#15)) - -(19) BroadcastExchange -Input [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#13] -Join condition: None - -(21) Project [codegen id : 4] -Output [7]: [i_brand#2, i_category#3, ss_sales_price#7, d_year#10, d_moy#11, s_store_name#14, s_company_name#15] -Input [9]: [i_brand#2, i_category#3, ss_store_sk#6, ss_sales_price#7, d_year#10, d_moy#11, s_store_sk#13, s_store_name#14, s_company_name#15] - -(22) HashAggregate [codegen id : 4] -Input [7]: [i_brand#2, i_category#3, ss_sales_price#7, d_year#10, d_moy#11, s_store_name#14, s_company_name#15] -Keys [6]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [1]: [sum#17] -Results [7]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum#18] - -(23) Exchange -Input [7]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum#18] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, 5), true, [id=#19] - -(24) HashAggregate [codegen id : 5] -Input [7]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum#18] -Keys [6]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11] -Functions [1]: [sum(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#7))#20] -Results [8]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#20,17,2) AS _w0#22] - -(25) Exchange -Input [8]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, 5), true, [id=#23] - -(26) Sort [codegen id : 6] -Input [8]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, _w0#22] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST, s_company_name#15 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], false, 0 - -(27) Window -Input [8]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#24], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10] - -(28) Project [codegen id : 7] -Output [8]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, avg_monthly_sales#24] -Input [9]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, _w0#22, avg_monthly_sales#24] - -(29) Exchange -Input [8]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, avg_monthly_sales#24] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#14, s_company_name#15, 5), true, [id=#25] - -(30) Sort [codegen id : 8] -Input [8]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, avg_monthly_sales#24] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST, s_company_name#15 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 - -(31) Window -Input [8]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, avg_monthly_sales#24] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] - -(32) Filter [codegen id : 23] -Input [9]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, avg_monthly_sales#24, rn#26] -Condition : (((((isnotnull(d_year#10) AND isnotnull(avg_monthly_sales#24)) AND (d_year#10 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CASE WHEN (avg_monthly_sales#24 > 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000)) AND isnotnull(rn#26)) - -(33) ReusedExchange [Reuses operator id: 23] -Output [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33] - -(34) HashAggregate [codegen id : 13] -Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33] -Keys [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32] -Functions [1]: [sum(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#7))#34] -Results [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#34,17,2) AS sum_sales#35] - -(35) Exchange -Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#35] -Arguments: hashpartitioning(i_category#27, i_brand#28, s_store_name#29, s_company_name#30, 5), true, [id=#36] - -(36) Sort [codegen id : 14] -Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#35] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST, s_company_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST], false, 0 - -(37) Window -Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#35] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, s_store_name#29, s_company_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(38) Filter [codegen id : 15] -Input [8]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#35, rn#37] -Condition : isnotnull(rn#37) - -(39) Project [codegen id : 15] -Output [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#35, rn#37] -Input [8]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#35, rn#37] - -(40) BroadcastExchange -Input [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] + 1)),false), [id=#38] - -(41) BroadcastHashJoin [codegen id : 23] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#26] -Right keys [5]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, (rn#37 + 1)] -Join condition: None - -(42) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] -Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#35, rn#37] - -(43) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] - -(44) Sort [codegen id : 21] -Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, s_store_name#41 ASC NULLS FIRST, s_company_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 - -(45) Window -Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#39, i_brand#40, s_store_name#41, s_company_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] - -(46) Filter [codegen id : 22] -Input [8]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] -Condition : isnotnull(rn#46) - -(47) Project [codegen id : 22] -Output [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] -Input [8]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] - -(48) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] - 1)),false), [id=#47] - -(49) BroadcastHashJoin [codegen id : 23] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#26] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, (rn#46 - 1)] -Join condition: None - -(50) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#48, sum_sales#45 AS nsum#49] -Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] - -(51) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, avg_monthly_sales#24, sum_sales#21, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, avg_monthly_sales#24, sum_sales#21, psum#48, nsum#49] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q47/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q47/simplified.txt deleted file mode 100644 index 66ccfa4a9..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q47/simplified.txt +++ /dev/null @@ -1,84 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (23) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Filter [d_year,avg_monthly_sales,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (8) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (7) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (6) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,d_year] #2 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_sold_date_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk,i_category,i_brand] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Filter [s_store_sk,s_store_name,s_company_name] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (15) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - Filter [rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (14) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #8 - WholeStageCodegen (13) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (22) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - Filter [rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (21) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q48/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q48/explain.txt deleted file mode 100644 index f317bcdf1..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q48/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* HashAggregate (32) -+- Exchange (31) - +- * HashAggregate (30) - +- * Project (29) - +- * BroadcastHashJoin Inner BuildRight (28) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.store (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.customer_demographics (10) - : +- BroadcastExchange (20) - : +- * Project (19) - : +- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet default.customer_address (16) - +- BroadcastExchange (27) - +- * Project (26) - +- * Filter (25) - +- * ColumnarToRow (24) - +- Scan parquet default.date_dim (23) - - -(1) Scan parquet default.store_sales -Output [7]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_net_profit#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_sold_date_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [7]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_net_profit#7] - -(3) Filter [codegen id : 5] -Input [7]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_net_profit#7] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_cdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_sold_date_sk#1)) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) AND ((((ss_net_profit#7 >= 0.00) AND (ss_net_profit#7 <= 2000.00)) OR ((ss_net_profit#7 >= 150.00) AND (ss_net_profit#7 <= 3000.00))) OR ((ss_net_profit#7 >= 50.00) AND (ss_net_profit#7 <= 25000.00)))) - -(4) Scan parquet default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#8] - -(6) Filter [codegen id : 1] -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(7) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#9] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#8] -Join condition: None - -(9) Project [codegen id : 5] -Output [6]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_net_profit#7] -Input [8]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_net_profit#7, s_store_sk#8] - -(10) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -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)))] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] - -(12) Filter [codegen id : 2] -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Condition : (isnotnull(cd_demo_sk#10) AND ((((cd_marital_status#11 = M) AND (cd_education_status#12 = 4 yr Degree)) OR ((cd_marital_status#11 = D) AND (cd_education_status#12 = 2 yr Degree))) OR ((cd_marital_status#11 = S) AND (cd_education_status#12 = College)))) - -(13) BroadcastExchange -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13] - -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join condition: ((((((cd_marital_status#11 = M) AND (cd_education_status#12 = 4 yr Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#11 = D) AND (cd_education_status#12 = 2 yr Degree)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#11 = S) AND (cd_education_status#12 = College)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) - -(15) Project [codegen id : 5] -Output [4]: [ss_sold_date_sk#1, ss_addr_sk#3, ss_quantity#5, ss_net_profit#7] -Input [9]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_net_profit#7, cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] - -(16) Scan parquet default.customer_address -Output [3]: [ca_address_sk#14, ca_state#15, ca_country#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [CO,OH,TX]),In(ca_state, [OR,MN,KY])),In(ca_state, [VA,CA,MS]))] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [3]: [ca_address_sk#14, ca_state#15, ca_country#16] - -(18) Filter [codegen id : 3] -Input [3]: [ca_address_sk#14, ca_state#15, ca_country#16] -Condition : (((isnotnull(ca_country#16) AND (ca_country#16 = United States)) AND isnotnull(ca_address_sk#14)) AND ((ca_state#15 IN (CO,OH,TX) OR ca_state#15 IN (OR,MN,KY)) OR ca_state#15 IN (VA,CA,MS))) - -(19) Project [codegen id : 3] -Output [2]: [ca_address_sk#14, ca_state#15] -Input [3]: [ca_address_sk#14, ca_state#15, ca_country#16] - -(20) BroadcastExchange -Input [2]: [ca_address_sk#14, ca_state#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#14] -Join condition: ((((ca_state#15 IN (CO,OH,TX) AND (ss_net_profit#7 >= 0.00)) AND (ss_net_profit#7 <= 2000.00)) OR ((ca_state#15 IN (OR,MN,KY) AND (ss_net_profit#7 >= 150.00)) AND (ss_net_profit#7 <= 3000.00))) OR ((ca_state#15 IN (VA,CA,MS) AND (ss_net_profit#7 >= 50.00)) AND (ss_net_profit#7 <= 25000.00))) - -(22) Project [codegen id : 5] -Output [2]: [ss_sold_date_sk#1, ss_quantity#5] -Input [6]: [ss_sold_date_sk#1, ss_addr_sk#3, ss_quantity#5, ss_net_profit#7, ca_address_sk#14, ca_state#15] - -(23) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#18, d_year#19] - -(25) Filter [codegen id : 4] -Input [2]: [d_date_sk#18, d_year#19] -Condition : ((isnotnull(d_year#19) AND (d_year#19 = 2001)) AND isnotnull(d_date_sk#18)) - -(26) Project [codegen id : 4] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_year#19] - -(27) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] - -(28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#18] -Join condition: None - -(29) Project [codegen id : 5] -Output [1]: [ss_quantity#5] -Input [3]: [ss_sold_date_sk#1, ss_quantity#5, d_date_sk#18] - -(30) HashAggregate [codegen id : 5] -Input [1]: [ss_quantity#5] -Keys: [] -Functions [1]: [partial_sum(cast(ss_quantity#5 as bigint))] -Aggregate Attributes [1]: [sum#21] -Results [1]: [sum#22] - -(31) Exchange -Input [1]: [sum#22] -Arguments: SinglePartition, true, [id=#23] - -(32) HashAggregate [codegen id : 6] -Input [1]: [sum#22] -Keys: [] -Functions [1]: [sum(cast(ss_quantity#5 as bigint))] -Aggregate Attributes [1]: [sum(cast(ss_quantity#5 as bigint))#24] -Results [1]: [sum(cast(ss_quantity#5 as bigint))#24 AS sum(ss_quantity)#25] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q48/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q48/simplified.txt deleted file mode 100644 index 710fbdd72..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q48/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -WholeStageCodegen (6) - HashAggregate [sum] [sum(cast(ss_quantity as bigint)),sum(ss_quantity),sum] - InputAdapter - Exchange #1 - WholeStageCodegen (5) - HashAggregate [ss_quantity] [sum,sum] - Project [ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_quantity] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_sold_date_sk,ss_addr_sk,ss_quantity,ss_net_profit] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_sold_date_sk,ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sold_date_sk,ss_sales_price,ss_net_profit] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [cd_demo_sk,cd_marital_status,cd_education_status] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [ca_address_sk,ca_state] - Filter [ca_country,ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q49/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q49/explain.txt deleted file mode 100644 index 8d10c1641..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q49/explain.txt +++ /dev/null @@ -1,433 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (78) -+- * HashAggregate (77) - +- Exchange (76) - +- * HashAggregate (75) - +- Union (74) - :- * Project (27) - : +- * Filter (26) - : +- Window (25) - : +- * Sort (24) - : +- Window (23) - : +- * Sort (22) - : +- Exchange (21) - : +- * HashAggregate (20) - : +- Exchange (19) - : +- * HashAggregate (18) - : +- * Project (17) - : +- * BroadcastHashJoin Inner BuildRight (16) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.web_sales (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet default.web_returns (5) - : +- BroadcastExchange (15) - : +- * Project (14) - : +- * Filter (13) - : +- * ColumnarToRow (12) - : +- Scan parquet default.date_dim (11) - :- * Project (50) - : +- * Filter (49) - : +- Window (48) - : +- * Sort (47) - : +- Window (46) - : +- * Sort (45) - : +- Exchange (44) - : +- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (31) - : : : +- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet default.catalog_sales (28) - : : +- BroadcastExchange (35) - : : +- * Filter (34) - : : +- * ColumnarToRow (33) - : : +- Scan parquet default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (73) - +- * Filter (72) - +- Window (71) - +- * Sort (70) - +- Window (69) - +- * Sort (68) - +- Exchange (67) - +- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * Project (60) - : +- * BroadcastHashJoin Inner BuildRight (59) - : :- * Project (54) - : : +- * Filter (53) - : : +- * ColumnarToRow (52) - : : +- Scan parquet default.store_sales (51) - : +- BroadcastExchange (58) - : +- * Filter (57) - : +- * ColumnarToRow (56) - : +- Scan parquet default.store_returns (55) - +- ReusedExchange (61) - - -(1) Scan parquet default.web_sales -Output [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, ws_net_profit#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, ws_net_profit#6] - -(3) Filter [codegen id : 3] -Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, ws_net_profit#6] -Condition : ((((((((isnotnull(ws_net_profit#6) AND isnotnull(ws_net_paid#5)) AND isnotnull(ws_quantity#4)) AND (ws_net_profit#6 > 1.00)) AND (ws_net_paid#5 > 0.00)) AND (ws_quantity#4 > 0)) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_item_sk#2)) AND isnotnull(ws_sold_date_sk#1)) - -(4) Project [codegen id : 3] -Output [5]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5] -Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, ws_net_profit#6] - -(5) Scan parquet default.web_returns -Output [4]: [wr_item_sk#7, wr_order_number#8, wr_return_quantity#9, wr_return_amt#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [wr_item_sk#7, wr_order_number#8, wr_return_quantity#9, wr_return_amt#10] - -(7) Filter [codegen id : 1] -Input [4]: [wr_item_sk#7, wr_order_number#8, wr_return_quantity#9, wr_return_amt#10] -Condition : (((isnotnull(wr_return_amt#10) AND (wr_return_amt#10 > 10000.00)) AND isnotnull(wr_order_number#8)) AND isnotnull(wr_item_sk#7)) - -(8) BroadcastExchange -Input [4]: [wr_item_sk#7, wr_order_number#8, wr_return_quantity#9, wr_return_amt#10] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#11] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [cast(ws_order_number#3 as bigint), cast(ws_item_sk#2 as bigint)] -Right keys [2]: [wr_order_number#8, wr_item_sk#7] -Join condition: None - -(10) Project [codegen id : 3] -Output [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_quantity#4, ws_net_paid#5, wr_return_quantity#9, wr_return_amt#10] -Input [9]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, wr_item_sk#7, wr_order_number#8, wr_return_quantity#9, wr_return_amt#10] - -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#12, d_year#13, d_moy#14] - -(13) Filter [codegen id : 2] -Input [3]: [d_date_sk#12, d_year#13, d_moy#14] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#14)) AND (d_year#13 = 2001)) AND (d_moy#14 = 12)) AND isnotnull(d_date_sk#12)) - -(14) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#14] - -(15) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#12] -Join condition: None - -(17) Project [codegen id : 3] -Output [5]: [ws_item_sk#2, ws_quantity#4, ws_net_paid#5, wr_return_quantity#9, wr_return_amt#10] -Input [7]: [ws_sold_date_sk#1, ws_item_sk#2, ws_quantity#4, ws_net_paid#5, wr_return_quantity#9, wr_return_amt#10, d_date_sk#12] - -(18) HashAggregate [codegen id : 3] -Input [5]: [ws_item_sk#2, ws_quantity#4, ws_net_paid#5, wr_return_quantity#9, wr_return_amt#10] -Keys [1]: [ws_item_sk#2] -Functions [4]: [partial_sum(cast(coalesce(wr_return_quantity#9, 0) as bigint)), partial_sum(cast(coalesce(ws_quantity#4, 0) as bigint)), partial_sum(coalesce(cast(wr_return_amt#10 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#5 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Results [7]: [ws_item_sk#2, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(19) Exchange -Input [7]: [ws_item_sk#2, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(ws_item_sk#2, 5), true, [id=#28] - -(20) HashAggregate [codegen id : 4] -Input [7]: [ws_item_sk#2, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [ws_item_sk#2] -Functions [4]: [sum(cast(coalesce(wr_return_quantity#9, 0) as bigint)), sum(cast(coalesce(ws_quantity#4, 0) as bigint)), sum(coalesce(cast(wr_return_amt#10 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#5 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(cast(coalesce(wr_return_quantity#9, 0) as bigint))#29, sum(cast(coalesce(ws_quantity#4, 0) as bigint))#30, sum(coalesce(cast(wr_return_amt#10 as decimal(12,2)), 0.00))#31, sum(coalesce(cast(ws_net_paid#5 as decimal(12,2)), 0.00))#32] -Results [3]: [ws_item_sk#2 AS item#33, CheckOverflow((promote_precision(cast(sum(cast(coalesce(wr_return_quantity#9, 0) as bigint))#29 as decimal(15,4))) / promote_precision(cast(sum(cast(coalesce(ws_quantity#4, 0) as bigint))#30 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#34, CheckOverflow((promote_precision(cast(sum(coalesce(cast(wr_return_amt#10 as decimal(12,2)), 0.00))#31 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ws_net_paid#5 as decimal(12,2)), 0.00))#32 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#35] - -(21) Exchange -Input [3]: [item#33, return_ratio#34, currency_ratio#35] -Arguments: SinglePartition, true, [id=#36] - -(22) Sort [codegen id : 5] -Input [3]: [item#33, return_ratio#34, currency_ratio#35] -Arguments: [return_ratio#34 ASC NULLS FIRST], false, 0 - -(23) Window -Input [3]: [item#33, return_ratio#34, currency_ratio#35] -Arguments: [rank(return_ratio#34) windowspecdefinition(return_ratio#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#37], [return_ratio#34 ASC NULLS FIRST] - -(24) Sort [codegen id : 6] -Input [4]: [item#33, return_ratio#34, currency_ratio#35, return_rank#37] -Arguments: [currency_ratio#35 ASC NULLS FIRST], false, 0 - -(25) Window -Input [4]: [item#33, return_ratio#34, currency_ratio#35, return_rank#37] -Arguments: [rank(currency_ratio#35) windowspecdefinition(currency_ratio#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#38], [currency_ratio#35 ASC NULLS FIRST] - -(26) Filter [codegen id : 7] -Input [5]: [item#33, return_ratio#34, currency_ratio#35, return_rank#37, currency_rank#38] -Condition : ((return_rank#37 <= 10) OR (currency_rank#38 <= 10)) - -(27) Project [codegen id : 7] -Output [5]: [web AS channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] -Input [5]: [item#33, return_ratio#34, currency_ratio#35, return_rank#37, currency_rank#38] - -(28) Scan parquet default.catalog_sales -Output [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44, cs_net_profit#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 10] -Input [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44, cs_net_profit#45] - -(30) Filter [codegen id : 10] -Input [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44, cs_net_profit#45] -Condition : ((((((((isnotnull(cs_net_profit#45) AND isnotnull(cs_net_paid#44)) AND isnotnull(cs_quantity#43)) AND (cs_net_profit#45 > 1.00)) AND (cs_net_paid#44 > 0.00)) AND (cs_quantity#43 > 0)) AND isnotnull(cs_order_number#42)) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_sold_date_sk#40)) - -(31) Project [codegen id : 10] -Output [5]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44] -Input [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44, cs_net_profit#45] - -(32) Scan parquet default.catalog_returns -Output [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 8] -Input [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] - -(34) Filter [codegen id : 8] -Input [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] -Condition : (((isnotnull(cr_return_amount#49) AND (cr_return_amount#49 > 10000.00)) AND isnotnull(cr_order_number#47)) AND isnotnull(cr_item_sk#46)) - -(35) BroadcastExchange -Input [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [id=#50] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#42, cs_item_sk#41] -Right keys [2]: [cr_order_number#47, cr_item_sk#46] -Join condition: None - -(37) Project [codegen id : 10] -Output [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_quantity#43, cs_net_paid#44, cr_return_quantity#48, cr_return_amount#49] -Input [9]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44, cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] - -(38) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#12] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#40] -Right keys [1]: [d_date_sk#12] -Join condition: None - -(40) Project [codegen id : 10] -Output [5]: [cs_item_sk#41, cs_quantity#43, cs_net_paid#44, cr_return_quantity#48, cr_return_amount#49] -Input [7]: [cs_sold_date_sk#40, cs_item_sk#41, cs_quantity#43, cs_net_paid#44, cr_return_quantity#48, cr_return_amount#49, d_date_sk#12] - -(41) HashAggregate [codegen id : 10] -Input [5]: [cs_item_sk#41, cs_quantity#43, cs_net_paid#44, cr_return_quantity#48, cr_return_amount#49] -Keys [1]: [cs_item_sk#41] -Functions [4]: [partial_sum(cast(coalesce(cr_return_quantity#48, 0) as bigint)), partial_sum(cast(coalesce(cs_quantity#43, 0) as bigint)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#44 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] -Results [7]: [cs_item_sk#41, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] - -(42) Exchange -Input [7]: [cs_item_sk#41, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] -Arguments: hashpartitioning(cs_item_sk#41, 5), true, [id=#63] - -(43) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#41, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] -Keys [1]: [cs_item_sk#41] -Functions [4]: [sum(cast(coalesce(cr_return_quantity#48, 0) as bigint)), sum(cast(coalesce(cs_quantity#43, 0) as bigint)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#44 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(cast(coalesce(cr_return_quantity#48, 0) as bigint))#64, sum(cast(coalesce(cs_quantity#43, 0) as bigint))#65, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#66, sum(coalesce(cast(cs_net_paid#44 as decimal(12,2)), 0.00))#67] -Results [3]: [cs_item_sk#41 AS item#68, CheckOverflow((promote_precision(cast(sum(cast(coalesce(cr_return_quantity#48, 0) as bigint))#64 as decimal(15,4))) / promote_precision(cast(sum(cast(coalesce(cs_quantity#43, 0) as bigint))#65 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#69, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#66 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#44 as decimal(12,2)), 0.00))#67 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#70] - -(44) Exchange -Input [3]: [item#68, return_ratio#69, currency_ratio#70] -Arguments: SinglePartition, true, [id=#71] - -(45) Sort [codegen id : 12] -Input [3]: [item#68, return_ratio#69, currency_ratio#70] -Arguments: [return_ratio#69 ASC NULLS FIRST], false, 0 - -(46) Window -Input [3]: [item#68, return_ratio#69, currency_ratio#70] -Arguments: [rank(return_ratio#69) windowspecdefinition(return_ratio#69 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#72], [return_ratio#69 ASC NULLS FIRST] - -(47) Sort [codegen id : 13] -Input [4]: [item#68, return_ratio#69, currency_ratio#70, return_rank#72] -Arguments: [currency_ratio#70 ASC NULLS FIRST], false, 0 - -(48) Window -Input [4]: [item#68, return_ratio#69, currency_ratio#70, return_rank#72] -Arguments: [rank(currency_ratio#70) windowspecdefinition(currency_ratio#70 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#73], [currency_ratio#70 ASC NULLS FIRST] - -(49) Filter [codegen id : 14] -Input [5]: [item#68, return_ratio#69, currency_ratio#70, return_rank#72, currency_rank#73] -Condition : ((return_rank#72 <= 10) OR (currency_rank#73 <= 10)) - -(50) Project [codegen id : 14] -Output [5]: [catalog AS channel#74, item#68, return_ratio#69, return_rank#72, currency_rank#73] -Input [5]: [item#68, return_ratio#69, currency_ratio#70, return_rank#72, currency_rank#73] - -(51) Scan parquet default.store_sales -Output [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79, ss_net_profit#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(52) ColumnarToRow [codegen id : 17] -Input [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79, ss_net_profit#80] - -(53) Filter [codegen id : 17] -Input [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79, ss_net_profit#80] -Condition : ((((((((isnotnull(ss_net_profit#80) AND isnotnull(ss_net_paid#79)) AND isnotnull(ss_quantity#78)) AND (ss_net_profit#80 > 1.00)) AND (ss_net_paid#79 > 0.00)) AND (ss_quantity#78 > 0)) AND isnotnull(ss_ticket_number#77)) AND isnotnull(ss_item_sk#76)) AND isnotnull(ss_sold_date_sk#75)) - -(54) Project [codegen id : 17] -Output [5]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79] -Input [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79, ss_net_profit#80] - -(55) Scan parquet default.store_returns -Output [4]: [sr_item_sk#81, sr_ticket_number#82, sr_return_quantity#83, sr_return_amt#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(56) ColumnarToRow [codegen id : 15] -Input [4]: [sr_item_sk#81, sr_ticket_number#82, sr_return_quantity#83, sr_return_amt#84] - -(57) Filter [codegen id : 15] -Input [4]: [sr_item_sk#81, sr_ticket_number#82, sr_return_quantity#83, sr_return_amt#84] -Condition : (((isnotnull(sr_return_amt#84) AND (sr_return_amt#84 > 10000.00)) AND isnotnull(sr_ticket_number#82)) AND isnotnull(sr_item_sk#81)) - -(58) BroadcastExchange -Input [4]: [sr_item_sk#81, sr_ticket_number#82, sr_return_quantity#83, sr_return_amt#84] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#85] - -(59) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#77 as bigint), cast(ss_item_sk#76 as bigint)] -Right keys [2]: [sr_ticket_number#82, sr_item_sk#81] -Join condition: None - -(60) Project [codegen id : 17] -Output [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_quantity#78, ss_net_paid#79, sr_return_quantity#83, sr_return_amt#84] -Input [9]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79, sr_item_sk#81, sr_ticket_number#82, sr_return_quantity#83, sr_return_amt#84] - -(61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#12] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#75] -Right keys [1]: [d_date_sk#12] -Join condition: None - -(63) Project [codegen id : 17] -Output [5]: [ss_item_sk#76, ss_quantity#78, ss_net_paid#79, sr_return_quantity#83, sr_return_amt#84] -Input [7]: [ss_sold_date_sk#75, ss_item_sk#76, ss_quantity#78, ss_net_paid#79, sr_return_quantity#83, sr_return_amt#84, d_date_sk#12] - -(64) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#76, ss_quantity#78, ss_net_paid#79, sr_return_quantity#83, sr_return_amt#84] -Keys [1]: [ss_item_sk#76] -Functions [4]: [partial_sum(cast(coalesce(sr_return_quantity#83, 0) as bigint)), partial_sum(cast(coalesce(ss_quantity#78, 0) as bigint)), partial_sum(coalesce(cast(sr_return_amt#84 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#79 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#86, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Results [7]: [ss_item_sk#76, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] - -(65) Exchange -Input [7]: [ss_item_sk#76, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Arguments: hashpartitioning(ss_item_sk#76, 5), true, [id=#98] - -(66) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#76, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Keys [1]: [ss_item_sk#76] -Functions [4]: [sum(cast(coalesce(sr_return_quantity#83, 0) as bigint)), sum(cast(coalesce(ss_quantity#78, 0) as bigint)), sum(coalesce(cast(sr_return_amt#84 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#79 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(cast(coalesce(sr_return_quantity#83, 0) as bigint))#99, sum(cast(coalesce(ss_quantity#78, 0) as bigint))#100, sum(coalesce(cast(sr_return_amt#84 as decimal(12,2)), 0.00))#101, sum(coalesce(cast(ss_net_paid#79 as decimal(12,2)), 0.00))#102] -Results [3]: [ss_item_sk#76 AS item#103, CheckOverflow((promote_precision(cast(sum(cast(coalesce(sr_return_quantity#83, 0) as bigint))#99 as decimal(15,4))) / promote_precision(cast(sum(cast(coalesce(ss_quantity#78, 0) as bigint))#100 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#104, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#84 as decimal(12,2)), 0.00))#101 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#79 as decimal(12,2)), 0.00))#102 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#105] - -(67) Exchange -Input [3]: [item#103, return_ratio#104, currency_ratio#105] -Arguments: SinglePartition, true, [id=#106] - -(68) Sort [codegen id : 19] -Input [3]: [item#103, return_ratio#104, currency_ratio#105] -Arguments: [return_ratio#104 ASC NULLS FIRST], false, 0 - -(69) Window -Input [3]: [item#103, return_ratio#104, currency_ratio#105] -Arguments: [rank(return_ratio#104) windowspecdefinition(return_ratio#104 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#107], [return_ratio#104 ASC NULLS FIRST] - -(70) Sort [codegen id : 20] -Input [4]: [item#103, return_ratio#104, currency_ratio#105, return_rank#107] -Arguments: [currency_ratio#105 ASC NULLS FIRST], false, 0 - -(71) Window -Input [4]: [item#103, return_ratio#104, currency_ratio#105, return_rank#107] -Arguments: [rank(currency_ratio#105) windowspecdefinition(currency_ratio#105 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#108], [currency_ratio#105 ASC NULLS FIRST] - -(72) Filter [codegen id : 21] -Input [5]: [item#103, return_ratio#104, currency_ratio#105, return_rank#107, currency_rank#108] -Condition : ((return_rank#107 <= 10) OR (currency_rank#108 <= 10)) - -(73) Project [codegen id : 21] -Output [5]: [store AS channel#109, item#103, return_ratio#104, return_rank#107, currency_rank#108] -Input [5]: [item#103, return_ratio#104, currency_ratio#105, return_rank#107, currency_rank#108] - -(74) Union - -(75) HashAggregate [codegen id : 22] -Input [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] -Keys [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] - -(76) Exchange -Input [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] -Arguments: hashpartitioning(channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38, 5), true, [id=#110] - -(77) HashAggregate [codegen id : 23] -Input [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] -Keys [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] - -(78) TakeOrderedAndProject -Input [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] -Arguments: 100, [channel#39 ASC NULLS FIRST, return_rank#37 ASC NULLS FIRST, currency_rank#38 ASC NULLS FIRST], [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q49/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q49/simplified.txt deleted file mode 100644 index c15f2394e..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q49/simplified.txt +++ /dev/null @@ -1,126 +0,0 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (23) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - Sort [return_ratio] - InputAdapter - Exchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(wr_return_quantity, 0) as bigint)),sum(cast(coalesce(ws_quantity, 0) as bigint)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [wr_return_amt,wr_order_number,wr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - Sort [return_ratio] - InputAdapter - Exchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(cr_return_quantity, 0) as bigint)),sum(cast(coalesce(cs_quantity, 0) as bigint)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Filter [cr_return_amount,cr_order_number,cr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - Sort [return_ratio] - InputAdapter - Exchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(sr_return_quantity, 0) as bigint)),sum(cast(coalesce(ss_quantity, 0) as bigint)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Filter [sr_return_amt,sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - InputAdapter - ReusedExchange [d_date_sk] #5 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q5/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q5/explain.txt deleted file mode 100644 index 15f0cda0b..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q5/explain.txt +++ /dev/null @@ -1,435 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (78) -+- * HashAggregate (77) - +- Exchange (76) - +- * HashAggregate (75) - +- * Expand (74) - +- Union (73) - :- * HashAggregate (25) - : +- Exchange (24) - : +- * HashAggregate (23) - : +- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- Union (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- * Project (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet default.store_returns (5) - : : +- BroadcastExchange (14) - : : +- * Project (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.date_dim (10) - : +- BroadcastExchange (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet default.store (17) - :- * HashAggregate (46) - : +- Exchange (45) - : +- * HashAggregate (44) - : +- * Project (43) - : +- * BroadcastHashJoin Inner BuildRight (42) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- Union (34) - : : : :- * Project (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.catalog_sales (26) - : : : +- * Project (33) - : : : +- * Filter (32) - : : : +- * ColumnarToRow (31) - : : : +- Scan parquet default.catalog_returns (30) - : : +- ReusedExchange (35) - : +- BroadcastExchange (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.catalog_page (38) - +- * HashAggregate (72) - +- Exchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- Union (60) - : : :- * Project (50) - : : : +- * Filter (49) - : : : +- * ColumnarToRow (48) - : : : +- Scan parquet default.web_sales (47) - : : +- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Filter (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet default.web_returns (51) - : : +- BroadcastExchange (57) - : : +- * Filter (56) - : : +- * ColumnarToRow (55) - : : +- Scan parquet default.web_sales (54) - : +- ReusedExchange (61) - +- BroadcastExchange (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet default.web_site (64) - - -(1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(3) Filter [codegen id : 1] -Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Condition : (isnotnull(cast(ss_sold_date_sk#1 as bigint)) AND isnotnull(cast(ss_store_sk#2 as bigint))) - -(4) Project [codegen id : 1] -Output [6]: [cast(ss_store_sk#2 as bigint) AS store_sk#5, cast(ss_sold_date_sk#1 as bigint) AS date_sk#6, ss_ext_sales_price#3 AS sales_price#7, ss_net_profit#4 AS profit#8, 0.00 AS return_amt#9, 0.00 AS net_loss#10] -Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(5) Scan parquet default.store_returns -Output [4]: [sr_returned_date_sk#11, sr_store_sk#12, sr_return_amt#13, sr_net_loss#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_returned_date_sk), IsNotNull(sr_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_returned_date_sk#11, sr_store_sk#12, sr_return_amt#13, sr_net_loss#14] - -(7) Filter [codegen id : 2] -Input [4]: [sr_returned_date_sk#11, sr_store_sk#12, sr_return_amt#13, sr_net_loss#14] -Condition : (isnotnull(sr_returned_date_sk#11) AND isnotnull(sr_store_sk#12)) - -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#15, sr_returned_date_sk#11 AS date_sk#16, 0.00 AS sales_price#17, 0.00 AS profit#18, sr_return_amt#13 AS return_amt#19, sr_net_loss#14 AS net_loss#20] -Input [4]: [sr_returned_date_sk#11, sr_store_sk#12, sr_return_amt#13, sr_net_loss#14] - -(9) Union - -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#21, d_date#22] - -(12) Filter [codegen id : 3] -Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 11192)) AND (d_date#22 <= 11206)) AND isnotnull(d_date_sk#21)) - -(13) Project [codegen id : 3] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_date#22] - -(14) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] - -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#6] -Right keys [1]: [cast(d_date_sk#21 as bigint)] -Join condition: None - -(16) Project [codegen id : 5] -Output [5]: [store_sk#5, sales_price#7, profit#8, return_amt#9, net_loss#10] -Input [7]: [store_sk#5, date_sk#6, sales_price#7, profit#8, return_amt#9, net_loss#10, d_date_sk#21] - -(17) Scan parquet default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#24, s_store_id#25] - -(19) Filter [codegen id : 4] -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(20) BroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [store_sk#5] -Right keys [1]: [cast(s_store_sk#24 as bigint)] -Join condition: None - -(22) Project [codegen id : 5] -Output [5]: [sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#25] -Input [7]: [store_sk#5, sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_sk#24, s_store_id#25] - -(23) HashAggregate [codegen id : 5] -Input [5]: [sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#25] -Keys [1]: [s_store_id#25] -Functions [4]: [partial_sum(UnscaledValue(sales_price#7)), partial_sum(UnscaledValue(return_amt#9)), partial_sum(UnscaledValue(profit#8)), partial_sum(UnscaledValue(net_loss#10))] -Aggregate Attributes [4]: [sum#27, sum#28, sum#29, sum#30] -Results [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] - -(24) Exchange -Input [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_id#25, 5), true, [id=#35] - -(25) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] -Keys [1]: [s_store_id#25] -Functions [4]: [sum(UnscaledValue(sales_price#7)), sum(UnscaledValue(return_amt#9)), sum(UnscaledValue(profit#8)), sum(UnscaledValue(net_loss#10))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#7))#36, sum(UnscaledValue(return_amt#9))#37, sum(UnscaledValue(profit#8))#38, sum(UnscaledValue(net_loss#10))#39] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#7))#36,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#9))#37,17,2) AS RETURNS#41, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#8))#38,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#10))#39,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#42, store channel AS channel#43, concat(store, s_store_id#25) AS id#44] - -(26) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#47, cs_net_profit#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(27) ColumnarToRow [codegen id : 7] -Input [4]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#47, cs_net_profit#48] - -(28) Filter [codegen id : 7] -Input [4]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#47, cs_net_profit#48] -Condition : (isnotnull(cs_sold_date_sk#45) AND isnotnull(cs_catalog_page_sk#46)) - -(29) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#46 AS page_sk#49, cs_sold_date_sk#45 AS date_sk#50, cs_ext_sales_price#47 AS sales_price#51, cs_net_profit#48 AS profit#52, 0.00 AS return_amt#53, 0.00 AS net_loss#54] -Input [4]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#47, cs_net_profit#48] - -(30) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#55, cr_catalog_page_sk#56, cr_return_amount#57, cr_net_loss#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 8] -Input [4]: [cr_returned_date_sk#55, cr_catalog_page_sk#56, cr_return_amount#57, cr_net_loss#58] - -(32) Filter [codegen id : 8] -Input [4]: [cr_returned_date_sk#55, cr_catalog_page_sk#56, cr_return_amount#57, cr_net_loss#58] -Condition : (isnotnull(cr_returned_date_sk#55) AND isnotnull(cr_catalog_page_sk#56)) - -(33) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#56 AS page_sk#59, cr_returned_date_sk#55 AS date_sk#60, 0.00 AS sales_price#61, 0.00 AS profit#62, cr_return_amount#57 AS return_amt#63, cr_net_loss#58 AS net_loss#64] -Input [4]: [cr_returned_date_sk#55, cr_catalog_page_sk#56, cr_return_amount#57, cr_net_loss#58] - -(34) Union - -(35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#21] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#50] -Right keys [1]: [d_date_sk#21] -Join condition: None - -(37) Project [codegen id : 11] -Output [5]: [page_sk#49, sales_price#51, profit#52, return_amt#53, net_loss#54] -Input [7]: [page_sk#49, date_sk#50, sales_price#51, profit#52, return_amt#53, net_loss#54, d_date_sk#21] - -(38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] - -(40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -Condition : isnotnull(cp_catalog_page_sk#65) - -(41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#67] - -(42) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#49] -Right keys [1]: [cp_catalog_page_sk#65] -Join condition: None - -(43) Project [codegen id : 11] -Output [5]: [sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_id#66] -Input [7]: [page_sk#49, sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_sk#65, cp_catalog_page_id#66] - -(44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_id#66] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [partial_sum(UnscaledValue(sales_price#51)), partial_sum(UnscaledValue(return_amt#53)), partial_sum(UnscaledValue(profit#52)), partial_sum(UnscaledValue(net_loss#54))] -Aggregate Attributes [4]: [sum#68, sum#69, sum#70, sum#71] -Results [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] - -(45) Exchange -Input [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(cp_catalog_page_id#66, 5), true, [id=#76] - -(46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [sum(UnscaledValue(sales_price#51)), sum(UnscaledValue(return_amt#53)), sum(UnscaledValue(profit#52)), sum(UnscaledValue(net_loss#54))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#51))#77, sum(UnscaledValue(return_amt#53))#78, sum(UnscaledValue(profit#52))#79, sum(UnscaledValue(net_loss#54))#80] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#51))#77,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#53))#78,17,2) AS RETURNS#82, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#52))#79,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#54))#80,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#66) AS id#85] - -(47) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] - -(49) Filter [codegen id : 13] -Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] -Condition : (isnotnull(cast(ws_sold_date_sk#86 as bigint)) AND isnotnull(ws_web_site_sk#87)) - -(50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#87 AS wsr_web_site_sk#90, cast(ws_sold_date_sk#86 as bigint) AS date_sk#91, ws_ext_sales_price#88 AS sales_price#92, ws_net_profit#89 AS profit#93, 0.00 AS return_amt#94, 0.00 AS net_loss#95] -Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] - -(51) Scan parquet default.web_returns -Output [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_returned_date_sk)] -ReadSchema: struct - -(52) ColumnarToRow [codegen id : 15] -Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] - -(53) Filter [codegen id : 15] -Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] -Condition : isnotnull(wr_returned_date_sk#96) - -(54) Scan parquet default.web_sales -Output [3]: [ws_item_sk#101, ws_web_site_sk#87, ws_order_number#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(55) ColumnarToRow [codegen id : 14] -Input [3]: [ws_item_sk#101, ws_web_site_sk#87, ws_order_number#102] - -(56) Filter [codegen id : 14] -Input [3]: [ws_item_sk#101, ws_web_site_sk#87, ws_order_number#102] -Condition : ((isnotnull(ws_item_sk#101) AND isnotnull(ws_order_number#102)) AND isnotnull(ws_web_site_sk#87)) - -(57) BroadcastExchange -Input [3]: [ws_item_sk#101, ws_web_site_sk#87, ws_order_number#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[2, int, false] as bigint)),false), [id=#103] - -(58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#97, wr_order_number#98] -Right keys [2]: [cast(ws_item_sk#101 as bigint), cast(ws_order_number#102 as bigint)] -Join condition: None - -(59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#87 AS wsr_web_site_sk#104, wr_returned_date_sk#96 AS date_sk#105, 0.00 AS sales_price#106, 0.00 AS profit#107, wr_return_amt#99 AS return_amt#108, wr_net_loss#100 AS net_loss#109] -Input [8]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100, ws_item_sk#101, ws_web_site_sk#87, ws_order_number#102] - -(60) Union - -(61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#21] - -(62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#91] -Right keys [1]: [cast(d_date_sk#21 as bigint)] -Join condition: None - -(63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95] -Input [7]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, d_date_sk#21] - -(64) Scan parquet default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#110, web_site_id#111] - -(66) Filter [codegen id : 17] -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) - -(67) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#112] - -(68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#90] -Right keys [1]: [web_site_sk#110] -Join condition: None - -(69) Project [codegen id : 18] -Output [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#111] -Input [7]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_sk#110, web_site_id#111] - -(70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#111] -Keys [1]: [web_site_id#111] -Functions [4]: [partial_sum(UnscaledValue(sales_price#92)), partial_sum(UnscaledValue(return_amt#94)), partial_sum(UnscaledValue(profit#93)), partial_sum(UnscaledValue(net_loss#95))] -Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] -Results [5]: [web_site_id#111, sum#117, sum#118, sum#119, sum#120] - -(71) Exchange -Input [5]: [web_site_id#111, sum#117, sum#118, sum#119, sum#120] -Arguments: hashpartitioning(web_site_id#111, 5), true, [id=#121] - -(72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#111, sum#117, sum#118, sum#119, sum#120] -Keys [1]: [web_site_id#111] -Functions [4]: [sum(UnscaledValue(sales_price#92)), sum(UnscaledValue(return_amt#94)), sum(UnscaledValue(profit#93)), sum(UnscaledValue(net_loss#95))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#92))#122, sum(UnscaledValue(return_amt#94))#123, sum(UnscaledValue(profit#93))#124, sum(UnscaledValue(net_loss#95))#125] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#92))#122,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#94))#123,17,2) AS RETURNS#127, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#93))#124,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#95))#125,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#128, web channel AS channel#129, concat(web_site, web_site_id#111) AS id#130] - -(73) Union - -(74) Expand [codegen id : 20] -Input [5]: [sales#40, RETURNS#41, profit#42, channel#43, id#44] -Arguments: [List(sales#40, returns#41, profit#42, channel#43, id#44, 0), List(sales#40, returns#41, profit#42, channel#43, null, 1), List(sales#40, returns#41, profit#42, null, null, 3)], [sales#40, returns#41, profit#42, channel#131, id#132, spark_grouping_id#133] - -(75) HashAggregate [codegen id : 20] -Input [6]: [sales#40, returns#41, profit#42, channel#131, id#132, spark_grouping_id#133] -Keys [3]: [channel#131, id#132, spark_grouping_id#133] -Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] -Aggregate Attributes [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Results [9]: [channel#131, id#132, spark_grouping_id#133, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] - -(76) Exchange -Input [9]: [channel#131, id#132, spark_grouping_id#133, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Arguments: hashpartitioning(channel#131, id#132, spark_grouping_id#133, 5), true, [id=#146] - -(77) HashAggregate [codegen id : 21] -Input [9]: [channel#131, id#132, spark_grouping_id#133, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Keys [3]: [channel#131, id#132, spark_grouping_id#133] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#147, sum(returns#41)#148, sum(profit#42)#149] -Results [5]: [channel#131, id#132, sum(sales#40)#147 AS sales#150, sum(returns#41)#148 AS returns#151, sum(profit#42)#149 AS profit#152] - -(78) TakeOrderedAndProject -Input [5]: [channel#131, id#132, sales#150, returns#151, profit#152] -Arguments: 100, [channel#131 ASC NULLS FIRST, id#132 ASC NULLS FIRST], [channel#131, id#132, sales#150, returns#151, profit#152] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q5/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q5/simplified.txt deleted file mode 100644 index 9b7cc3360..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q5/simplified.txt +++ /dev/null @@ -1,123 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (21) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (20) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,RETURNS,profit,channel,id,sum,sum,sum,sum] - InputAdapter - Exchange [s_store_id] #2 - WholeStageCodegen (5) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] - Filter [ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - WholeStageCodegen (2) - Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] - Filter [sr_returned_date_sk,sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_store_sk,sr_return_amt,sr_net_loss] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,RETURNS,profit,channel,id,sum,sum,sum,sum] - InputAdapter - Exchange [cp_catalog_page_id] #5 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] - Filter [cs_sold_date_sk,cs_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit] - WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] - Filter [cr_returned_date_sk,cr_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_catalog_page_sk,cr_return_amount,cr_net_loss] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - Filter [cp_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,RETURNS,profit,channel,id,sum,sum,sum,sum] - InputAdapter - Exchange [web_site_id] #7 - WholeStageCodegen (18) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) - Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] - Filter [ws_sold_date_sk,ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_web_site_sk,ws_ext_sales_price,ws_net_profit] - WholeStageCodegen (15) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Filter [wr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_returned_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) - Filter [ws_item_sk,ws_order_number,ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (17) - Filter [web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_site [web_site_sk,web_site_id] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q50/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q50/explain.txt deleted file mode 100644 index e083affa7..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q50/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.store_returns (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.store (10) - : +- BroadcastExchange (19) - : +- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet default.date_dim (16) - +- BroadcastExchange (26) - +- * Project (25) - +- * Filter (24) - +- * ColumnarToRow (23) - +- Scan parquet default.date_dim (22) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5] - -(3) Filter [codegen id : 5] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5] -Condition : ((((isnotnull(ss_ticket_number#5) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_sold_date_sk#1)) - -(4) Scan parquet default.store_returns -Output [4]: [sr_returned_date_sk#6, sr_item_sk#7, sr_customer_sk#8, sr_ticket_number#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [sr_returned_date_sk#6, sr_item_sk#7, sr_customer_sk#8, sr_ticket_number#9] - -(6) Filter [codegen id : 1] -Input [4]: [sr_returned_date_sk#6, sr_item_sk#7, sr_customer_sk#8, sr_ticket_number#9] -Condition : (((isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#7)) AND isnotnull(sr_customer_sk#8)) AND isnotnull(sr_returned_date_sk#6)) - -(7) BroadcastExchange -Input [4]: [sr_returned_date_sk#6, sr_item_sk#7, sr_customer_sk#8, sr_ticket_number#9] -Arguments: HashedRelationBroadcastMode(List(input[3, bigint, false], input[1, bigint, false], input[2, bigint, false]),false), [id=#10] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [3]: [cast(ss_ticket_number#5 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_customer_sk#3 as bigint)] -Right keys [3]: [sr_ticket_number#9, sr_item_sk#7, sr_customer_sk#8] -Join condition: None - -(9) Project [codegen id : 5] -Output [3]: [ss_sold_date_sk#1, ss_store_sk#4, sr_returned_date_sk#6] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, sr_returned_date_sk#6, sr_item_sk#7, sr_customer_sk#8, sr_ticket_number#9] - -(10) Scan parquet default.store -Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] - -(12) Filter [codegen id : 2] -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Condition : isnotnull(s_store_sk#11) - -(13) BroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] - -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join condition: None - -(15) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#1, sr_returned_date_sk#6, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Input [14]: [ss_sold_date_sk#1, ss_store_sk#4, sr_returned_date_sk#6, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] - -(16) Scan parquet default.date_dim -Output [1]: [d_date_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [1]: [d_date_sk#23] - -(18) Filter [codegen id : 3] -Input [1]: [d_date_sk#23] -Condition : isnotnull(d_date_sk#23) - -(19) BroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] - -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#23] -Join condition: None - -(21) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#1, sr_returned_date_sk#6, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Input [13]: [ss_sold_date_sk#1, sr_returned_date_sk#6, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#23] - -(22) Scan parquet default.date_dim -Output [3]: [d_date_sk#25, d_year#26, d_moy#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(23) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#25, d_year#26, d_moy#27] - -(24) Filter [codegen id : 4] -Input [3]: [d_date_sk#25, d_year#26, d_moy#27] -Condition : ((((isnotnull(d_year#26) AND isnotnull(d_moy#27)) AND (d_year#26 = 2001)) AND (d_moy#27 = 8)) AND isnotnull(d_date_sk#25)) - -(25) Project [codegen id : 4] -Output [1]: [d_date_sk#25] -Input [3]: [d_date_sk#25, d_year#26, d_moy#27] - -(26) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] - -(27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_returned_date_sk#6] -Right keys [1]: [cast(d_date_sk#25 as bigint)] -Join condition: None - -(28) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#1, sr_returned_date_sk#6, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Input [13]: [ss_sold_date_sk#1, sr_returned_date_sk#6, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#25] - -(29) HashAggregate [codegen id : 5] -Input [12]: [ss_sold_date_sk#1, sr_returned_date_sk#6, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Functions [5]: [partial_sum(cast(CASE WHEN ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum#29, sum#30, sum#31, sum#32, sum#33] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#34, sum#35, sum#36, sum#37, sum#38] - -(30) Exchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#34, sum#35, sum#36, sum#37, sum#38] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), true, [id=#39] - -(31) HashAggregate [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#34, sum#35, sum#36, sum#37, sum#38] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Functions [5]: [sum(cast(CASE WHEN ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum(cast(CASE WHEN ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint))#40, sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint))#41, sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint))#42, sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint))#43, sum(cast(CASE WHEN ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))#44] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum(cast(CASE WHEN ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint))#40 AS 30 days #45, sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint))#41 AS 31 - 60 days #46, sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint))#42 AS 61 - 90 days #47, sum(cast(CASE WHEN (((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint))#43 AS 91 - 120 days #48, sum(cast(CASE WHEN ((sr_returned_date_sk#6 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))#44 AS >120 days #49] - -(32) TakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #45, 31 - 60 days #46, 61 - 90 days #47, 91 - 120 days #48, >120 days #49] -Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #45, 31 - 60 days #46, 61 - 90 days #47, 91 - 120 days #48, >120 days #49] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q50/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q50/simplified.txt deleted file mode 100644 index 43e777385..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q50/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) - HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] [sum(cast(CASE WHEN ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 30) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 60) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 90) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 120) THEN 1 ELSE 0 END as bigint)),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - InputAdapter - Exchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - WholeStageCodegen (5) - HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_sold_date_sk,ss_store_sk,sr_returned_date_sk] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - Filter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [sr_ticket_number,sr_item_sk,sr_customer_sk,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q51/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q51/explain.txt deleted file mode 100644 index b391d90a3..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q51/explain.txt +++ /dev/null @@ -1,228 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- Window (39) - +- * Sort (38) - +- Exchange (37) - +- * Project (36) - +- SortMergeJoin FullOuter (35) - :- * Sort (19) - : +- Exchange (18) - : +- * Project (17) - : +- Window (16) - : +- * Sort (15) - : +- Exchange (14) - : +- * HashAggregate (13) - : +- Exchange (12) - : +- * HashAggregate (11) - : +- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.web_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (34) - +- Exchange (33) - +- * Project (32) - +- Window (31) - +- * Sort (30) - +- Exchange (29) - +- * HashAggregate (28) - +- Exchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Filter (22) - : +- * ColumnarToRow (21) - : +- Scan parquet default.store_sales (20) - +- ReusedExchange (23) - - -(1) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_sales_price#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_sales_price#3] - -(3) Filter [codegen id : 2] -Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_sales_price#3] -Condition : (isnotnull(ws_item_sk#2) AND isnotnull(ws_sold_date_sk#1)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(7) Project [codegen id : 1] -Output [2]: [d_date_sk#4, d_date#5] -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] - -(8) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(10) Project [codegen id : 2] -Output [3]: [ws_item_sk#2, ws_sales_price#3, d_date#5] -Input [5]: [ws_sold_date_sk#1, ws_item_sk#2, ws_sales_price#3, d_date_sk#4, d_date#5] - -(11) HashAggregate [codegen id : 2] -Input [3]: [ws_item_sk#2, ws_sales_price#3, d_date#5] -Keys [2]: [ws_item_sk#2, d_date#5] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#3))] -Aggregate Attributes [1]: [sum#8] -Results [3]: [ws_item_sk#2, d_date#5, sum#9] - -(12) Exchange -Input [3]: [ws_item_sk#2, d_date#5, sum#9] -Arguments: hashpartitioning(ws_item_sk#2, d_date#5, 5), true, [id=#10] - -(13) HashAggregate [codegen id : 3] -Input [3]: [ws_item_sk#2, d_date#5, sum#9] -Keys [2]: [ws_item_sk#2, d_date#5] -Functions [1]: [sum(UnscaledValue(ws_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#3))#11] -Results [4]: [ws_item_sk#2 AS item_sk#12, d_date#5, MakeDecimal(sum(UnscaledValue(ws_sales_price#3))#11,17,2) AS _w0#13, ws_item_sk#2] - -(14) Exchange -Input [4]: [item_sk#12, d_date#5, _w0#13, ws_item_sk#2] -Arguments: hashpartitioning(ws_item_sk#2, 5), true, [id=#14] - -(15) Sort [codegen id : 4] -Input [4]: [item_sk#12, d_date#5, _w0#13, ws_item_sk#2] -Arguments: [ws_item_sk#2 ASC NULLS FIRST, d_date#5 ASC NULLS FIRST], false, 0 - -(16) Window -Input [4]: [item_sk#12, d_date#5, _w0#13, ws_item_sk#2] -Arguments: [sum(_w0#13) windowspecdefinition(ws_item_sk#2, d_date#5 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#15], [ws_item_sk#2], [d_date#5 ASC NULLS FIRST] - -(17) Project [codegen id : 5] -Output [3]: [item_sk#12, d_date#5, cume_sales#15] -Input [5]: [item_sk#12, d_date#5, _w0#13, ws_item_sk#2, cume_sales#15] - -(18) Exchange -Input [3]: [item_sk#12, d_date#5, cume_sales#15] -Arguments: hashpartitioning(item_sk#12, d_date#5, 5), true, [id=#16] - -(19) Sort [codegen id : 6] -Input [3]: [item_sk#12, d_date#5, cume_sales#15] -Arguments: [item_sk#12 ASC NULLS FIRST, d_date#5 ASC NULLS FIRST], false, 0 - -(20) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#17, ss_item_sk#18, ss_sales_price#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(21) ColumnarToRow [codegen id : 8] -Input [3]: [ss_sold_date_sk#17, ss_item_sk#18, ss_sales_price#19] - -(22) Filter [codegen id : 8] -Input [3]: [ss_sold_date_sk#17, ss_item_sk#18, ss_sales_price#19] -Condition : (isnotnull(ss_item_sk#18) AND isnotnull(ss_sold_date_sk#17)) - -(23) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#20, d_date#21] - -(24) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#20] -Join condition: None - -(25) Project [codegen id : 8] -Output [3]: [ss_item_sk#18, ss_sales_price#19, d_date#21] -Input [5]: [ss_sold_date_sk#17, ss_item_sk#18, ss_sales_price#19, d_date_sk#20, d_date#21] - -(26) HashAggregate [codegen id : 8] -Input [3]: [ss_item_sk#18, ss_sales_price#19, d_date#21] -Keys [2]: [ss_item_sk#18, d_date#21] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#19))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [ss_item_sk#18, d_date#21, sum#23] - -(27) Exchange -Input [3]: [ss_item_sk#18, d_date#21, sum#23] -Arguments: hashpartitioning(ss_item_sk#18, d_date#21, 5), true, [id=#24] - -(28) HashAggregate [codegen id : 9] -Input [3]: [ss_item_sk#18, d_date#21, sum#23] -Keys [2]: [ss_item_sk#18, d_date#21] -Functions [1]: [sum(UnscaledValue(ss_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#19))#25] -Results [4]: [ss_item_sk#18 AS item_sk#26, d_date#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#19))#25,17,2) AS _w0#27, ss_item_sk#18] - -(29) Exchange -Input [4]: [item_sk#26, d_date#21, _w0#27, ss_item_sk#18] -Arguments: hashpartitioning(ss_item_sk#18, 5), true, [id=#28] - -(30) Sort [codegen id : 10] -Input [4]: [item_sk#26, d_date#21, _w0#27, ss_item_sk#18] -Arguments: [ss_item_sk#18 ASC NULLS FIRST, d_date#21 ASC NULLS FIRST], false, 0 - -(31) Window -Input [4]: [item_sk#26, d_date#21, _w0#27, ss_item_sk#18] -Arguments: [sum(_w0#27) windowspecdefinition(ss_item_sk#18, d_date#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#29], [ss_item_sk#18], [d_date#21 ASC NULLS FIRST] - -(32) Project [codegen id : 11] -Output [3]: [item_sk#26, d_date#21, cume_sales#29] -Input [5]: [item_sk#26, d_date#21, _w0#27, ss_item_sk#18, cume_sales#29] - -(33) Exchange -Input [3]: [item_sk#26, d_date#21, cume_sales#29] -Arguments: hashpartitioning(item_sk#26, d_date#21, 5), true, [id=#30] - -(34) Sort [codegen id : 12] -Input [3]: [item_sk#26, d_date#21, cume_sales#29] -Arguments: [item_sk#26 ASC NULLS FIRST, d_date#21 ASC NULLS FIRST], false, 0 - -(35) SortMergeJoin -Left keys [2]: [item_sk#12, d_date#5] -Right keys [2]: [item_sk#26, d_date#21] -Join condition: None - -(36) Project [codegen id : 13] -Output [4]: [CASE WHEN isnotnull(item_sk#12) THEN item_sk#12 ELSE item_sk#26 END AS item_sk#31, CASE WHEN isnotnull(d_date#5) THEN d_date#5 ELSE d_date#21 END AS d_date#32, cume_sales#15 AS web_sales#33, cume_sales#29 AS store_sales#34] -Input [6]: [item_sk#12, d_date#5, cume_sales#15, item_sk#26, d_date#21, cume_sales#29] - -(37) Exchange -Input [4]: [item_sk#31, d_date#32, web_sales#33, store_sales#34] -Arguments: hashpartitioning(item_sk#31, 5), true, [id=#35] - -(38) Sort [codegen id : 14] -Input [4]: [item_sk#31, d_date#32, web_sales#33, store_sales#34] -Arguments: [item_sk#31 ASC NULLS FIRST, d_date#32 ASC NULLS FIRST], false, 0 - -(39) Window -Input [4]: [item_sk#31, d_date#32, web_sales#33, store_sales#34] -Arguments: [max(web_sales#33) windowspecdefinition(item_sk#31, d_date#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#36, max(store_sales#34) windowspecdefinition(item_sk#31, d_date#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#37], [item_sk#31], [d_date#32 ASC NULLS FIRST] - -(40) Filter [codegen id : 15] -Input [6]: [item_sk#31, d_date#32, web_sales#33, store_sales#34, web_cumulative#36, store_cumulative#37] -Condition : ((isnotnull(web_cumulative#36) AND isnotnull(store_cumulative#37)) AND (web_cumulative#36 > store_cumulative#37)) - -(41) TakeOrderedAndProject -Input [6]: [item_sk#31, d_date#32, web_sales#33, store_sales#34, web_cumulative#36, store_cumulative#37] -Arguments: 100, [item_sk#31 ASC NULLS FIRST, d_date#32 ASC NULLS FIRST], [item_sk#31, d_date#32, web_sales#33, store_sales#34, web_cumulative#36, store_cumulative#37] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q51/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q51/simplified.txt deleted file mode 100644 index 17f35514f..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q51/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (15) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (14) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk] #1 - WholeStageCodegen (13) - Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] - InputAdapter - SortMergeJoin [item_sk,d_date,item_sk,d_date] - WholeStageCodegen (6) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - InputAdapter - Exchange [ws_item_sk,d_date] #4 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_sales_price] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - WholeStageCodegen (12) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #6 - WholeStageCodegen (11) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (10) - Sort [ss_item_sk,d_date] - InputAdapter - Exchange [ss_item_sk] #7 - WholeStageCodegen (9) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - InputAdapter - Exchange [ss_item_sk,d_date] #8 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_sales_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q52/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q52/explain.txt deleted file mode 100644 index 47235253f..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q52/explain.txt +++ /dev/null @@ -1,122 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.date_dim (1) - : +- BroadcastExchange (8) - : +- * Filter (7) - : +- * ColumnarToRow (6) - : +- Scan parquet default.store_sales (5) - +- BroadcastExchange (15) - +- * Project (14) - +- * Filter (13) - +- * ColumnarToRow (12) - +- Scan parquet default.item (11) - - -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] - -(3) Filter [codegen id : 3] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(4) Project [codegen id : 3] -Output [2]: [d_date_sk#1, d_year#2] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] - -(5) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] - -(7) Filter [codegen id : 1] -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Condition : (isnotnull(ss_sold_date_sk#4) AND isnotnull(ss_item_sk#5)) - -(8) BroadcastExchange -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#5, ss_ext_sales_price#6] -Input [5]: [d_date_sk#1, d_year#2, ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] - -(11) Scan parquet default.item -Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] - -(13) Filter [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] -Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 1)) AND isnotnull(i_item_sk#8)) - -(14) Project [codegen id : 2] -Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] - -(15) BroadcastExchange -Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#8] -Join condition: None - -(17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#6, i_brand_id#9, i_brand#10] -Input [6]: [d_year#2, ss_item_sk#5, ss_ext_sales_price#6, i_item_sk#8, i_brand_id#9, i_brand#10] - -(18) HashAggregate [codegen id : 3] -Input [4]: [d_year#2, ss_ext_sales_price#6, i_brand_id#9, i_brand#10] -Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum#13] -Results [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] - -(19) Exchange -Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] -Arguments: hashpartitioning(d_year#2, i_brand#10, i_brand_id#9, 5), true, [id=#15] - -(20) HashAggregate [codegen id : 4] -Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] -Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#16] -Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#16,17,2) AS ext_price#19] - -(21) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#17, brand#18, ext_price#19] -Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q52/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q52/simplified.txt deleted file mode 100644 index ba48ad5b6..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q52/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject [d_year,ext_price,brand_id,brand] - WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - InputAdapter - Exchange [d_year,i_brand,i_brand_id] #1 - WholeStageCodegen (3) - HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [d_date_sk,d_year] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q53/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q53/explain.txt deleted file mode 100644 index 249f74e73..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q53/explain.txt +++ /dev/null @@ -1,180 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * Filter (30) - +- Window (29) - +- * Sort (28) - +- Exchange (27) - +- * HashAggregate (26) - +- Exchange (25) - +- * HashAggregate (24) - +- * Project (23) - +- * BroadcastHashJoin Inner BuildRight (22) - :- * Project (17) - : +- * BroadcastHashJoin Inner BuildRight (16) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet default.store_sales (5) - : +- BroadcastExchange (15) - : +- * Project (14) - : +- * Filter (13) - : +- * ColumnarToRow (12) - : +- Scan parquet default.date_dim (11) - +- BroadcastExchange (21) - +- * Filter (20) - +- * ColumnarToRow (19) - +- Scan parquet default.store (18) - - -(1) Scan parquet default.item -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)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -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)) - -(4) Project [codegen id : 4] -Output [2]: [i_item_sk#1, i_manufact_id#5] -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] - -(5) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#10, ss_item_sk#11, ss_store_sk#12, ss_sales_price#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_sold_date_sk#10, ss_item_sk#11, ss_store_sk#12, ss_sales_price#13] - -(7) Filter [codegen id : 1] -Input [4]: [ss_sold_date_sk#10, ss_item_sk#11, ss_store_sk#12, ss_sales_price#13] -Condition : ((isnotnull(ss_item_sk#11) AND isnotnull(ss_sold_date_sk#10)) AND isnotnull(ss_store_sk#12)) - -(8) BroadcastExchange -Input [4]: [ss_sold_date_sk#10, ss_item_sk#11, ss_store_sk#12, ss_sales_price#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#14] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#11] -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_sold_date_sk#10, ss_store_sk#12, ss_sales_price#13] -Input [6]: [i_item_sk#1, i_manufact_id#5, ss_sold_date_sk#10, ss_item_sk#11, ss_store_sk#12, ss_sales_price#13] - -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1211,1205,1201,1206,1210,1207,1202,1209,1203,1208,1204]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] - -(13) Filter [codegen id : 2] -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET (1200,1211,1205,1201,1206,1210,1207,1202,1209,1203,1208,1204) AND isnotnull(d_date_sk#15)) - -(14) Project [codegen id : 2] -Output [2]: [d_date_sk#15, d_qoy#17] -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] - -(15) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#15] -Join condition: None - -(17) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#12, ss_sales_price#13, d_qoy#17] -Input [6]: [i_manufact_id#5, ss_sold_date_sk#10, ss_store_sk#12, ss_sales_price#13, d_date_sk#15, d_qoy#17] - -(18) Scan parquet default.store -Output [1]: [s_store_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#19] - -(20) Filter [codegen id : 3] -Input [1]: [s_store_sk#19] -Condition : isnotnull(s_store_sk#19) - -(21) BroadcastExchange -Input [1]: [s_store_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#12] -Right keys [1]: [s_store_sk#19] -Join condition: None - -(23) Project [codegen id : 4] -Output [3]: [i_manufact_id#5, ss_sales_price#13, d_qoy#17] -Input [5]: [i_manufact_id#5, ss_store_sk#12, ss_sales_price#13, d_qoy#17, s_store_sk#19] - -(24) HashAggregate [codegen id : 4] -Input [3]: [i_manufact_id#5, ss_sales_price#13, d_qoy#17] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [i_manufact_id#5, d_qoy#17, sum#22] - -(25) Exchange -Input [3]: [i_manufact_id#5, d_qoy#17, sum#22] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), true, [id=#23] - -(26) HashAggregate [codegen id : 5] -Input [3]: [i_manufact_id#5, d_qoy#17, sum#22] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#13))#24] -Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#13))#24,17,2) AS sum_sales#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#13))#24,17,2) AS _w0#26] - -(27) Exchange -Input [3]: [i_manufact_id#5, sum_sales#25, _w0#26] -Arguments: hashpartitioning(i_manufact_id#5, 5), true, [id=#27] - -(28) Sort [codegen id : 6] -Input [3]: [i_manufact_id#5, sum_sales#25, _w0#26] -Arguments: [i_manufact_id#5 ASC NULLS FIRST], false, 0 - -(29) Window -Input [3]: [i_manufact_id#5, sum_sales#25, _w0#26] -Arguments: [avg(_w0#26) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#28], [i_manufact_id#5] - -(30) Filter [codegen id : 7] -Input [4]: [i_manufact_id#5, sum_sales#25, _w0#26, avg_quarterly_sales#28] -Condition : (CASE WHEN (avg_quarterly_sales#28 > 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#25 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#28 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#28 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000) - -(31) Project [codegen id : 7] -Output [3]: [i_manufact_id#5, sum_sales#25, avg_quarterly_sales#28] -Input [4]: [i_manufact_id#5, sum_sales#25, _w0#26, avg_quarterly_sales#28] - -(32) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#25, avg_quarterly_sales#28] -Arguments: 100, [avg_quarterly_sales#28 ASC NULLS FIRST, sum_sales#25 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#25, avg_quarterly_sales#28] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q53/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q53/simplified.txt deleted file mode 100644 index cd55b60d2..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q53/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (7) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (6) - Sort [i_manufact_id] - InputAdapter - Exchange [i_manufact_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_manufact_id,d_qoy] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_sold_date_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [i_item_sk,i_manufact_id] - Filter [i_category,i_class,i_brand,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date_sk,d_qoy] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q54/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q54/explain.txt deleted file mode 100644 index 2e93e2605..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q54/explain.txt +++ /dev/null @@ -1,459 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (64) -+- * HashAggregate (63) - +- Exchange (62) - +- * HashAggregate (61) - +- * HashAggregate (60) - +- Exchange (59) - +- * HashAggregate (58) - +- * Project (57) - +- * BroadcastHashJoin Inner BuildRight (56) - :- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (44) - : : +- * BroadcastHashJoin Inner BuildRight (43) - : : :- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * HashAggregate (32) - : : : : +- Exchange (31) - : : : : +- * HashAggregate (30) - : : : : +- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (16) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : : :- Union (9) - : : : : : : : :- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- * Project (8) - : : : : : : : +- * Filter (7) - : : : : : : : +- * ColumnarToRow (6) - : : : : : : : +- Scan parquet default.web_sales (5) - : : : : : : +- BroadcastExchange (14) - : : : : : : +- * Project (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Project (20) - : : : : : +- * Filter (19) - : : : : : +- * ColumnarToRow (18) - : : : : : +- Scan parquet default.date_dim (17) - : : : : +- BroadcastExchange (27) - : : : : +- * Filter (26) - : : : : +- * ColumnarToRow (25) - : : : : +- Scan parquet default.customer (24) - : : : +- BroadcastExchange (36) - : : : +- * Filter (35) - : : : +- * ColumnarToRow (34) - : : : +- Scan parquet default.store_sales (33) - : : +- BroadcastExchange (42) - : : +- * Filter (41) - : : +- * ColumnarToRow (40) - : : +- Scan parquet default.customer_address (39) - : +- BroadcastExchange (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.store (45) - +- BroadcastExchange (55) - +- * Project (54) - +- * Filter (53) - +- * ColumnarToRow (52) - +- Scan parquet default.date_dim (51) - - -(1) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3] - -(3) Filter [codegen id : 1] -Input [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3] -Condition : ((isnotnull(cs_item_sk#3) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_bill_customer_sk#2)) - -(4) Project [codegen id : 1] -Output [3]: [cs_sold_date_sk#1 AS sold_date_sk#4, cs_bill_customer_sk#2 AS customer_sk#5, cs_item_sk#3 AS item_sk#6] -Input [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3] - -(5) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#7, ws_item_sk#8, ws_bill_customer_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [3]: [ws_sold_date_sk#7, ws_item_sk#8, ws_bill_customer_sk#9] - -(7) Filter [codegen id : 2] -Input [3]: [ws_sold_date_sk#7, ws_item_sk#8, ws_bill_customer_sk#9] -Condition : ((isnotnull(ws_item_sk#8) AND isnotnull(ws_sold_date_sk#7)) AND isnotnull(ws_bill_customer_sk#9)) - -(8) Project [codegen id : 2] -Output [3]: [ws_sold_date_sk#7 AS sold_date_sk#10, ws_bill_customer_sk#9 AS customer_sk#11, ws_item_sk#8 AS item_sk#12] -Input [3]: [ws_sold_date_sk#7, ws_item_sk#8, ws_bill_customer_sk#9] - -(9) Union - -(10) Scan parquet default.item -Output [3]: [i_item_sk#13, i_class#14, i_category#15] -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)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 3] -Input [3]: [i_item_sk#13, i_class#14, i_category#15] - -(12) Filter [codegen id : 3] -Input [3]: [i_item_sk#13, i_class#14, i_category#15] -Condition : ((((isnotnull(i_category#15) AND isnotnull(i_class#14)) AND (i_category#15 = Women)) AND (i_class#14 = maternity)) AND isnotnull(i_item_sk#13)) - -(13) Project [codegen id : 3] -Output [1]: [i_item_sk#13] -Input [3]: [i_item_sk#13, i_class#14, i_category#15] - -(14) BroadcastExchange -Input [1]: [i_item_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_sk#6] -Right keys [1]: [i_item_sk#13] -Join condition: None - -(16) Project [codegen id : 6] -Output [2]: [sold_date_sk#4, customer_sk#5] -Input [4]: [sold_date_sk#4, customer_sk#5, item_sk#6, i_item_sk#13] - -(17) Scan parquet default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(19) Filter [codegen id : 4] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(20) Project [codegen id : 4] -Output [1]: [d_date_sk#17] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(21) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] - -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sold_date_sk#4] -Right keys [1]: [d_date_sk#17] -Join condition: None - -(23) Project [codegen id : 6] -Output [1]: [customer_sk#5] -Input [3]: [sold_date_sk#4, customer_sk#5, d_date_sk#17] - -(24) Scan parquet default.customer -Output [2]: [c_customer_sk#21, c_current_addr_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 5] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] - -(26) Filter [codegen id : 5] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) - -(27) BroadcastExchange -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [customer_sk#5] -Right keys [1]: [c_customer_sk#21] -Join condition: None - -(29) Project [codegen id : 6] -Output [2]: [c_customer_sk#21, c_current_addr_sk#22] -Input [3]: [customer_sk#5, c_customer_sk#21, c_current_addr_sk#22] - -(30) HashAggregate [codegen id : 6] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Keys [2]: [c_customer_sk#21, c_current_addr_sk#22] -Functions: [] -Aggregate Attributes: [] -Results [2]: [c_customer_sk#21, c_current_addr_sk#22] - -(31) Exchange -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Arguments: hashpartitioning(c_customer_sk#21, c_current_addr_sk#22, 5), true, [id=#24] - -(32) HashAggregate [codegen id : 11] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Keys [2]: [c_customer_sk#21, c_current_addr_sk#22] -Functions: [] -Aggregate Attributes: [] -Results [2]: [c_customer_sk#21, c_current_addr_sk#22] - -(33) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 7] -Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] - -(35) Filter [codegen id : 7] -Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] -Condition : (isnotnull(ss_customer_sk#26) AND isnotnull(ss_sold_date_sk#25)) - -(36) BroadcastExchange -Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#28] - -(37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_customer_sk#21] -Right keys [1]: [ss_customer_sk#26] -Join condition: None - -(38) Project [codegen id : 11] -Output [4]: [c_customer_sk#21, c_current_addr_sk#22, ss_sold_date_sk#25, ss_ext_sales_price#27] -Input [5]: [c_customer_sk#21, c_current_addr_sk#22, ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] - -(39) Scan parquet default.customer_address -Output [3]: [ca_address_sk#29, ca_county#30, ca_state#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] -ReadSchema: struct - -(40) ColumnarToRow [codegen id : 8] -Input [3]: [ca_address_sk#29, ca_county#30, ca_state#31] - -(41) Filter [codegen id : 8] -Input [3]: [ca_address_sk#29, ca_county#30, ca_state#31] -Condition : ((isnotnull(ca_address_sk#29) AND isnotnull(ca_county#30)) AND isnotnull(ca_state#31)) - -(42) BroadcastExchange -Input [3]: [ca_address_sk#29, ca_county#30, ca_state#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] - -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#22] -Right keys [1]: [ca_address_sk#29] -Join condition: None - -(44) Project [codegen id : 11] -Output [5]: [c_customer_sk#21, ss_sold_date_sk#25, ss_ext_sales_price#27, ca_county#30, ca_state#31] -Input [7]: [c_customer_sk#21, c_current_addr_sk#22, ss_sold_date_sk#25, ss_ext_sales_price#27, ca_address_sk#29, ca_county#30, ca_state#31] - -(45) Scan parquet default.store -Output [2]: [s_county#33, s_state#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [s_county#33, s_state#34] - -(47) Filter [codegen id : 9] -Input [2]: [s_county#33, s_state#34] -Condition : (isnotnull(s_county#33) AND isnotnull(s_state#34)) - -(48) BroadcastExchange -Input [2]: [s_county#33, s_state#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [id=#35] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [ca_county#30, ca_state#31] -Right keys [2]: [s_county#33, s_state#34] -Join condition: None - -(50) Project [codegen id : 11] -Output [3]: [c_customer_sk#21, ss_sold_date_sk#25, ss_ext_sales_price#27] -Input [7]: [c_customer_sk#21, ss_sold_date_sk#25, ss_ext_sales_price#27, ca_county#30, ca_state#31, s_county#33, s_state#34] - -(51) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_month_seq#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) ColumnarToRow [codegen id : 10] -Input [2]: [d_date_sk#17, d_month_seq#36] - -(53) Filter [codegen id : 10] -Input [2]: [d_date_sk#17, d_month_seq#36] -Condition : (((isnotnull(d_month_seq#36) AND (d_month_seq#36 >= Subquery scalar-subquery#37, [id=#38])) AND (d_month_seq#36 <= Subquery scalar-subquery#39, [id=#40])) AND isnotnull(d_date_sk#17)) - -(54) Project [codegen id : 10] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_month_seq#36] - -(55) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] - -(56) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#17] -Join condition: None - -(57) Project [codegen id : 11] -Output [2]: [c_customer_sk#21, ss_ext_sales_price#27] -Input [4]: [c_customer_sk#21, ss_sold_date_sk#25, ss_ext_sales_price#27, d_date_sk#17] - -(58) HashAggregate [codegen id : 11] -Input [2]: [c_customer_sk#21, ss_ext_sales_price#27] -Keys [1]: [c_customer_sk#21] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#42] -Results [2]: [c_customer_sk#21, sum#43] - -(59) Exchange -Input [2]: [c_customer_sk#21, sum#43] -Arguments: hashpartitioning(c_customer_sk#21, 5), true, [id=#44] - -(60) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#21, sum#43] -Keys [1]: [c_customer_sk#21] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#45] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#45,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#46] - -(61) HashAggregate [codegen id : 12] -Input [1]: [segment#46] -Keys [1]: [segment#46] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#47] -Results [2]: [segment#46, count#48] - -(62) Exchange -Input [2]: [segment#46, count#48] -Arguments: hashpartitioning(segment#46, 5), true, [id=#49] - -(63) HashAggregate [codegen id : 13] -Input [2]: [segment#46, count#48] -Keys [1]: [segment#46] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#50] -Results [3]: [segment#46, count(1)#50 AS num_customers#51, (segment#46 * 50) AS segment_base#52] - -(64) TakeOrderedAndProject -Input [3]: [segment#46, num_customers#51, segment_base#52] -Arguments: 100, [segment#46 ASC NULLS FIRST, num_customers#51 ASC NULLS FIRST], [segment#46, num_customers#51, segment_base#52] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* HashAggregate (71) -+- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * Filter (67) - +- * ColumnarToRow (66) - +- Scan parquet default.date_dim (65) - - -(65) Scan parquet default.date_dim -Output [3]: [d_month_seq#36, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(66) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#36, d_year#18, d_moy#19] - -(67) Filter [codegen id : 1] -Input [3]: [d_month_seq#36, d_year#18, d_moy#19] -Condition : (((isnotnull(d_year#18) AND isnotnull(d_moy#19)) AND (d_year#18 = 1998)) AND (d_moy#19 = 12)) - -(68) Project [codegen id : 1] -Output [1]: [(d_month_seq#36 + 1) AS (d_month_seq + 1)#53] -Input [3]: [d_month_seq#36, d_year#18, d_moy#19] - -(69) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#53] -Keys [1]: [(d_month_seq + 1)#53] -Functions: [] -Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#53] - -(70) Exchange -Input [1]: [(d_month_seq + 1)#53] -Arguments: hashpartitioning((d_month_seq + 1)#53, 5), true, [id=#54] - -(71) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#53] -Keys [1]: [(d_month_seq + 1)#53] -Functions: [] -Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#53] - -Subquery:2 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] -* HashAggregate (78) -+- Exchange (77) - +- * HashAggregate (76) - +- * Project (75) - +- * Filter (74) - +- * ColumnarToRow (73) - +- Scan parquet default.date_dim (72) - - -(72) Scan parquet default.date_dim -Output [3]: [d_month_seq#36, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(73) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#36, d_year#18, d_moy#19] - -(74) Filter [codegen id : 1] -Input [3]: [d_month_seq#36, d_year#18, d_moy#19] -Condition : (((isnotnull(d_year#18) AND isnotnull(d_moy#19)) AND (d_year#18 = 1998)) AND (d_moy#19 = 12)) - -(75) Project [codegen id : 1] -Output [1]: [(d_month_seq#36 + 3) AS (d_month_seq + 3)#55] -Input [3]: [d_month_seq#36, d_year#18, d_moy#19] - -(76) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#55] -Keys [1]: [(d_month_seq + 3)#55] -Functions: [] -Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#55] - -(77) Exchange -Input [1]: [(d_month_seq + 3)#55] -Arguments: hashpartitioning((d_month_seq + 3)#55, 5), true, [id=#56] - -(78) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#55] -Keys [1]: [(d_month_seq + 3)#55] -Functions: [] -Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#55] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q54/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q54/simplified.txt deleted file mode 100644 index dd92c52db..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q54/simplified.txt +++ /dev/null @@ -1,121 +0,0 @@ -TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (13) - HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - InputAdapter - Exchange [segment] #1 - WholeStageCodegen (12) - HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - InputAdapter - Exchange [c_customer_sk] #2 - WholeStageCodegen (11) - HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] - Project [c_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_sk,ss_sold_date_sk,ss_ext_sales_price] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ss_sold_date_sk,ss_ext_sales_price,ca_county,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,ss_sold_date_sk,ss_ext_sales_price] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [c_customer_sk,c_current_addr_sk] - InputAdapter - Exchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (6) - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - BroadcastHashJoin [customer_sk,c_customer_sk] - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - Filter [cs_item_sk,cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - WholeStageCodegen (2) - Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] - Filter [ws_item_sk,ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [i_item_sk] - Filter [i_category,i_class,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Filter [ss_customer_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Filter [ca_address_sk,ca_county,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Filter [s_county,s_state] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_county,s_state] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (10) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #1 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 1)] - InputAdapter - Exchange [(d_month_seq + 1)] #11 - WholeStageCodegen (1) - HashAggregate [(d_month_seq + 1)] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - Subquery #2 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 3)] - InputAdapter - Exchange [(d_month_seq + 3)] #12 - WholeStageCodegen (1) - HashAggregate [(d_month_seq + 3)] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q55/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q55/explain.txt deleted file mode 100644 index 99c79d204..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q55/explain.txt +++ /dev/null @@ -1,122 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.date_dim (1) - : +- BroadcastExchange (8) - : +- * Filter (7) - : +- * ColumnarToRow (6) - : +- Scan parquet default.store_sales (5) - +- BroadcastExchange (15) - +- * Project (14) - +- * Filter (13) - +- * ColumnarToRow (12) - +- Scan parquet default.item (11) - - -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] - -(3) Filter [codegen id : 3] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) - -(4) Project [codegen id : 3] -Output [1]: [d_date_sk#1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] - -(5) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] - -(7) Filter [codegen id : 1] -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Condition : (isnotnull(ss_sold_date_sk#4) AND isnotnull(ss_item_sk#5)) - -(8) BroadcastExchange -Input [3]: [ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] -Join condition: None - -(10) Project [codegen id : 3] -Output [2]: [ss_item_sk#5, ss_ext_sales_price#6] -Input [4]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_ext_sales_price#6] - -(11) Scan parquet default.item -Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] - -(13) Filter [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] -Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 28)) AND isnotnull(i_item_sk#8)) - -(14) Project [codegen id : 2] -Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] - -(15) BroadcastExchange -Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#8] -Join condition: None - -(17) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#6, i_brand_id#9, i_brand#10] -Input [5]: [ss_item_sk#5, ss_ext_sales_price#6, i_item_sk#8, i_brand_id#9, i_brand#10] - -(18) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#6, i_brand_id#9, i_brand#10] -Keys [2]: [i_brand#10, i_brand_id#9] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum#13] -Results [3]: [i_brand#10, i_brand_id#9, sum#14] - -(19) Exchange -Input [3]: [i_brand#10, i_brand_id#9, sum#14] -Arguments: hashpartitioning(i_brand#10, i_brand_id#9, 5), true, [id=#15] - -(20) HashAggregate [codegen id : 4] -Input [3]: [i_brand#10, i_brand_id#9, sum#14] -Keys [2]: [i_brand#10, i_brand_id#9] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#16] -Results [3]: [i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#16,17,2) AS ext_price#19] - -(21) TakeOrderedAndProject -Input [3]: [brand_id#17, brand#18, ext_price#19] -Arguments: 100, [ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [brand_id#17, brand#18, ext_price#19] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q55/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q55/simplified.txt deleted file mode 100644 index bb2ef8e82..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q55/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject [ext_price,brand_id,brand] - WholeStageCodegen (4) - HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - InputAdapter - Exchange [i_brand,i_brand_id] #1 - WholeStageCodegen (3) - HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q56/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q56/explain.txt deleted file mode 100644 index d0d647211..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q56/explain.txt +++ /dev/null @@ -1,378 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) - :- * HashAggregate (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer_address (11) - : +- BroadcastExchange (27) - : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : :- * Filter (20) - : : +- * ColumnarToRow (19) - : : +- Scan parquet default.item (18) - : +- BroadcastExchange (25) - : +- * Project (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (41) - : : +- * BroadcastHashJoin Inner BuildRight (40) - : : :- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.catalog_sales (33) - : : : +- ReusedExchange (36) - : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) - - -(1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] - -(3) Filter [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_item_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_moy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -Condition : ((((isnotnull(d_year#6) AND isnotnull(d_moy#7)) AND (d_year#6 = 2001)) AND (d_moy#7 = 2)) AND isnotnull(d_date_sk#5)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] - -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(10) Project [codegen id : 5] -Output [3]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, d_date_sk#5] - -(11) Scan parquet default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] - -(13) Filter [codegen id : 2] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(14) Project [codegen id : 2] -Output [1]: [ca_address_sk#9] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] - -(15) BroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(17) Project [codegen id : 5] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#4] -Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] - -(18) Scan parquet default.item -Output [2]: [i_item_sk#12, i_item_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] - -(20) Filter [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] -Condition : isnotnull(i_item_sk#12) - -(21) Scan parquet default.item -Output [2]: [i_item_id#13, i_color#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate,blanched,burnished])] -ReadSchema: struct - -(22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#13, i_color#14] - -(23) Filter [codegen id : 3] -Input [2]: [i_item_id#13, i_color#14] -Condition : i_color#14 IN (slate,blanched,burnished) - -(24) Project [codegen id : 3] -Output [1]: [i_item_id#13 AS i_item_id#13#15] -Input [2]: [i_item_id#13, i_color#14] - -(25) BroadcastExchange -Input [1]: [i_item_id#13#15] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#16] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_id#13] -Right keys [1]: [i_item_id#13#15] -Join condition: None - -(27) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] - -(28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(29) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#4, i_item_id#13] -Input [4]: [ss_item_sk#2, ss_ext_sales_price#4, i_item_sk#12, i_item_id#13] - -(30) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#4, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [1]: [sum#18] -Results [2]: [i_item_id#13, sum#19] - -(31) Exchange -Input [2]: [i_item_id#13, sum#19] -Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#20] - -(32) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#13, sum#19] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#4))#21] -Results [2]: [i_item_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] - -(33) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] - -(35) Filter [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Condition : ((isnotnull(cs_sold_date_sk#23) AND isnotnull(cs_bill_addr_sk#24)) AND isnotnull(cs_item_sk#25)) - -(36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(38) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_date_sk#5] - -(39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#9] - -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(41) Project [codegen id : 11] -Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#9] - -(42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#12, i_item_id#13] - -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#13] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#12, i_item_id#13] - -(45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#13, sum#28] - -(46) Exchange -Input [2]: [i_item_id#13, sum#28] -Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#29] - -(47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#13, sum#28] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#30] -Results [2]: [i_item_id#13, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] - -(48) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] - -(50) Filter [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Condition : ((isnotnull(ws_sold_date_sk#32) AND isnotnull(ws_bill_addr_sk#34)) AND isnotnull(ws_item_sk#33)) - -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, d_date_sk#5] - -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#9] - -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#9] - -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#12, i_item_id#13] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#13] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#12, i_item_id#13] - -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#36] -Results [2]: [i_item_id#13, sum#37] - -(61) Exchange -Input [2]: [i_item_id#13, sum#37] -Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#38] - -(62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#13, sum#37] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#39] -Results [2]: [i_item_id#13, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] - -(63) Union - -(64) HashAggregate [codegen id : 19] -Input [2]: [i_item_id#13, total_sales#22] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(total_sales#22)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_item_id#13, sum#43, isEmpty#44] - -(65) Exchange -Input [3]: [i_item_id#13, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#45] - -(66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#13, sum#43, isEmpty#44] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(total_sales#22)] -Aggregate Attributes [1]: [sum(total_sales#22)#46] -Results [2]: [i_item_id#13, sum(total_sales#22)#46 AS total_sales#47] - -(67) TakeOrderedAndProject -Input [2]: [i_item_id#13, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_item_id#13, total_sales#47] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q56/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q56/simplified.txt deleted file mode 100644 index 5825c6f6e..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q56/simplified.txt +++ /dev/null @@ -1,101 +0,0 @@ -TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (20) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [i_item_id] - Filter [i_color] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_id,i_color] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q57/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q57/explain.txt deleted file mode 100644 index 4e1123185..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q57/explain.txt +++ /dev/null @@ -1,278 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Filter (32) - : : +- Window (31) - : : +- * Sort (30) - : : +- Exchange (29) - : : +- * Project (28) - : : +- Window (27) - : : +- * Sort (26) - : : +- Exchange (25) - : : +- * HashAggregate (24) - : : +- Exchange (23) - : : +- * HashAggregate (22) - : : +- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.catalog_sales (4) - : : : +- BroadcastExchange (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (19) - : : +- * Filter (18) - : : +- * ColumnarToRow (17) - : : +- Scan parquet default.call_center (16) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- * Filter (38) - : +- Window (37) - : +- * Sort (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- ReusedExchange (33) - +- BroadcastExchange (48) - +- * Project (47) - +- * Filter (46) - +- Window (45) - +- * Sort (44) - +- ReusedExchange (43) - - -(1) Scan parquet default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] - -(3) Filter [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) - -(4) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#4, cs_call_center_sk#5, cs_item_sk#6, cs_sales_price#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [cs_sold_date_sk#4, cs_call_center_sk#5, cs_item_sk#6, cs_sales_price#7] - -(6) Filter [codegen id : 1] -Input [4]: [cs_sold_date_sk#4, cs_call_center_sk#5, cs_item_sk#6, cs_sales_price#7] -Condition : ((isnotnull(cs_item_sk#6) AND isnotnull(cs_sold_date_sk#4)) AND isnotnull(cs_call_center_sk#5)) - -(7) BroadcastExchange -Input [4]: [cs_sold_date_sk#4, cs_call_center_sk#5, cs_item_sk#6, cs_sales_price#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, false] as bigint)),false), [id=#8] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#6] -Join condition: None - -(9) Project [codegen id : 4] -Output [5]: [i_brand#2, i_category#3, cs_sold_date_sk#4, cs_call_center_sk#5, cs_sales_price#7] -Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_sold_date_sk#4, cs_call_center_sk#5, cs_item_sk#6, cs_sales_price#7] - -(10) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] - -(12) Filter [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) - -(13) BroadcastExchange -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join condition: None - -(15) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, cs_call_center_sk#5, cs_sales_price#7, d_year#10, d_moy#11] -Input [8]: [i_brand#2, i_category#3, cs_sold_date_sk#4, cs_call_center_sk#5, cs_sales_price#7, d_date_sk#9, d_year#10, d_moy#11] - -(16) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#13, cc_name#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [2]: [cc_call_center_sk#13, cc_name#14] - -(18) Filter [codegen id : 3] -Input [2]: [cc_call_center_sk#13, cc_name#14] -Condition : (isnotnull(cc_call_center_sk#13) AND isnotnull(cc_name#14)) - -(19) BroadcastExchange -Input [2]: [cc_call_center_sk#13, cc_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#5] -Right keys [1]: [cc_call_center_sk#13] -Join condition: None - -(21) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, cs_sales_price#7, d_year#10, d_moy#11, cc_name#14] -Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#5, cs_sales_price#7, d_year#10, d_moy#11, cc_call_center_sk#13, cc_name#14] - -(22) HashAggregate [codegen id : 4] -Input [6]: [i_brand#2, i_category#3, cs_sales_price#7, d_year#10, d_moy#11, cc_name#14] -Keys [5]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#7))] -Aggregate Attributes [1]: [sum#16] -Results [6]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum#17] - -(23) Exchange -Input [6]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum#17] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, 5), true, [id=#18] - -(24) HashAggregate [codegen id : 5] -Input [6]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum#17] -Keys [5]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11] -Functions [1]: [sum(UnscaledValue(cs_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#7))#19] -Results [7]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#7))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(cs_sales_price#7))#19,17,2) AS _w0#21] - -(25) Exchange -Input [7]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#14, d_year#10, 5), true, [id=#22] - -(26) Sort [codegen id : 6] -Input [7]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, _w0#21] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], false, 0 - -(27) Window -Input [7]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#3, i_brand#2, cc_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#3, i_brand#2, cc_name#14, d_year#10] - -(28) Project [codegen id : 7] -Output [7]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, avg_monthly_sales#23] -Input [8]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, _w0#21, avg_monthly_sales#23] - -(29) Exchange -Input [7]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, avg_monthly_sales#23] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#14, 5), true, [id=#24] - -(30) Sort [codegen id : 8] -Input [7]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, avg_monthly_sales#23] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 - -(31) Window -Input [7]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, avg_monthly_sales#23] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#25], [i_category#3, i_brand#2, cc_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] - -(32) Filter [codegen id : 23] -Input [8]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, avg_monthly_sales#23, rn#25] -Condition : (((((isnotnull(d_year#10) AND isnotnull(avg_monthly_sales#23)) AND (d_year#10 = 1999)) AND (avg_monthly_sales#23 > 0.000000)) AND (CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000)) AND isnotnull(rn#25)) - -(33) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31] - -(34) HashAggregate [codegen id : 13] -Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31] -Keys [5]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30] -Functions [1]: [sum(UnscaledValue(cs_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#7))#32] -Results [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, MakeDecimal(sum(UnscaledValue(cs_sales_price#7))#32,17,2) AS sum_sales#33] - -(35) Exchange -Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#33] -Arguments: hashpartitioning(i_category#26, i_brand#27, cc_name#28, 5), true, [id=#34] - -(36) Sort [codegen id : 14] -Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#33] -Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST], false, 0 - -(37) Window -Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#33] -Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#26, i_brand#27, cc_name#28, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#26, i_brand#27, cc_name#28], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] - -(38) Filter [codegen id : 15] -Input [7]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#33, rn#35] -Condition : isnotnull(rn#35) - -(39) Project [codegen id : 15] -Output [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#33, rn#35] -Input [7]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#33, rn#35] - -(40) BroadcastExchange -Input [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#33, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] + 1)),false), [id=#36] - -(41) BroadcastHashJoin [codegen id : 23] -Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#25] -Right keys [4]: [i_category#26, i_brand#27, cc_name#28, (rn#35 + 1)] -Join condition: None - -(42) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, avg_monthly_sales#23, rn#25, sum_sales#33] -Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, avg_monthly_sales#23, rn#25, i_category#26, i_brand#27, cc_name#28, sum_sales#33, rn#35] - -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] - -(44) Sort [codegen id : 21] -Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] -Arguments: [i_category#37 ASC NULLS FIRST, i_brand#38 ASC NULLS FIRST, cc_name#39 ASC NULLS FIRST, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST], false, 0 - -(45) Window -Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] -Arguments: [rank(d_year#40, d_moy#41) windowspecdefinition(i_category#37, i_brand#38, cc_name#39, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#37, i_brand#38, cc_name#39], [d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST] - -(46) Filter [codegen id : 22] -Input [7]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42, rn#43] -Condition : isnotnull(rn#43) - -(47) Project [codegen id : 22] -Output [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] -Input [7]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42, rn#43] - -(48) BroadcastExchange -Input [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] - 1)),false), [id=#44] - -(49) BroadcastHashJoin [codegen id : 23] -Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#25] -Right keys [4]: [i_category#37, i_brand#38, cc_name#39, (rn#43 - 1)] -Join condition: None - -(50) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#45, sum_sales#42 AS nsum#46] -Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, avg_monthly_sales#23, rn#25, sum_sales#33, i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] - -(51) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, avg_monthly_sales#23, sum_sales#20, psum#45, nsum#46] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, avg_monthly_sales#23, sum_sales#20, psum#45, nsum#46] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q57/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q57/simplified.txt deleted file mode 100644 index 432441bb7..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q57/simplified.txt +++ /dev/null @@ -1,84 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (23) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Filter [d_year,avg_monthly_sales,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (8) - Sort [i_category,i_brand,cc_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (7) - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (6) - Sort [i_category,i_brand,cc_name,d_year] - InputAdapter - Exchange [i_category,i_brand,cc_name,d_year] #2 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_brand,cc_name,d_year,d_moy] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_sold_date_sk,cs_call_center_sk,cs_sales_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - Filter [i_item_sk,i_category,i_brand] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_sold_date_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_call_center_sk,cs_item_sk,cs_sales_price] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Filter [cc_call_center_sk,cc_name] - ColumnarToRow - InputAdapter - Scan parquet default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (15) - Project [i_category,i_brand,cc_name,sum_sales,rn] - Filter [rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (14) - Sort [i_category,i_brand,cc_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (13) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (22) - Project [i_category,i_brand,cc_name,sum_sales,rn] - Filter [rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (21) - Sort [i_category,i_brand,cc_name,d_year,d_moy] - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q58/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q58/explain.txt deleted file mode 100644 index 40a6836ae..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q58/explain.txt +++ /dev/null @@ -1,477 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Filter (26) - : : +- * HashAggregate (25) - : : +- Exchange (24) - : : +- * HashAggregate (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin LeftSemi BuildRight (18) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (17) - : : +- * Project (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.date_dim (13) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (30) - : +- BroadcastExchange (43) - : +- * Project (42) - : +- * BroadcastHashJoin LeftSemi BuildRight (41) - : :- * Filter (35) - : : +- * ColumnarToRow (34) - : : +- Scan parquet default.date_dim (33) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.date_dim (36) - +- BroadcastExchange (76) - +- * Filter (75) - +- * HashAggregate (74) - +- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet default.web_sales (53) - : +- ReusedExchange (56) - +- BroadcastExchange (69) - +- * Project (68) - +- * BroadcastHashJoin LeftSemi BuildRight (67) - :- * Filter (61) - : +- * ColumnarToRow (60) - : +- Scan parquet default.date_dim (59) - +- BroadcastExchange (66) - +- * Project (65) - +- * Filter (64) - +- * ColumnarToRow (63) - +- Scan parquet default.date_dim (62) - - -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] - -(3) Filter [codegen id : 4] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) - -(4) Scan parquet default.item -Output [2]: [i_item_sk#4, i_item_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] - -(6) Filter [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] -Condition : (isnotnull(i_item_sk#4) AND isnotnull(i_item_id#5)) - -(7) BroadcastExchange -Input [2]: [i_item_sk#4, i_item_id#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#4] -Join condition: None - -(9) Project [codegen id : 4] -Output [3]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#5] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_item_id#5] - -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#7, d_date#8] - -(12) Filter [codegen id : 3] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) - -(13) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] - -(15) Filter [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = Subquery scalar-subquery#10, [id=#11])) - -(16) Project [codegen id : 2] -Output [1]: [d_date#8 AS d_date#8#12] -Input [2]: [d_date#8, d_week_seq#9] - -(17) BroadcastExchange -Input [1]: [d_date#8#12] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] - -(18) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#12] -Join condition: None - -(19) Project [codegen id : 3] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] - -(20) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#7] -Join condition: None - -(22) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#3, i_item_id#5] -Input [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#5, d_date_sk#7] - -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#3, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#5, sum#16] - -(24) Exchange -Input [2]: [i_item_id#5, sum#16] -Arguments: hashpartitioning(i_item_id#5, 5), true, [id=#17] - -(25) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#5, sum#16] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [2]: [i_item_id#5 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS ss_item_rev#20] - -(26) Filter [codegen id : 15] -Input [2]: [item_id#19, ss_item_rev#20] -Condition : isnotnull(ss_item_rev#20) - -(27) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#21, cs_item_sk#22, cs_ext_sales_price#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_sold_date_sk#21, cs_item_sk#22, cs_ext_sales_price#23] - -(29) Filter [codegen id : 8] -Input [3]: [cs_sold_date_sk#21, cs_item_sk#22, cs_ext_sales_price#23] -Condition : (isnotnull(cs_item_sk#22) AND isnotnull(cs_sold_date_sk#21)) - -(30) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#4] -Join condition: None - -(32) Project [codegen id : 8] -Output [3]: [cs_sold_date_sk#21, cs_ext_sales_price#23, i_item_id#5] -Input [5]: [cs_sold_date_sk#21, cs_item_sk#22, cs_ext_sales_price#23, i_item_sk#4, i_item_id#5] - -(33) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#7, d_date#8] - -(35) Filter [codegen id : 7] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) - -(36) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(37) ColumnarToRow [codegen id : 6] -Input [2]: [d_date#8, d_week_seq#9] - -(38) Filter [codegen id : 6] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) - -(39) Project [codegen id : 6] -Output [1]: [d_date#8 AS d_date#8#24] -Input [2]: [d_date#8, d_week_seq#9] - -(40) BroadcastExchange -Input [1]: [d_date#8#24] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] - -(41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#24] -Join condition: None - -(42) Project [codegen id : 7] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] - -(43) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] - -(44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#7] -Join condition: None - -(45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#23, i_item_id#5] -Input [4]: [cs_sold_date_sk#21, cs_ext_sales_price#23, i_item_id#5, d_date_sk#7] - -(46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#23, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#5, sum#28] - -(47) Exchange -Input [2]: [i_item_id#5, sum#28] -Arguments: hashpartitioning(i_item_id#5, 5), true, [id=#29] - -(48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#5, sum#28] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#30] -Results [2]: [i_item_id#5 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#30,17,2) AS cs_item_rev#32] - -(49) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) - -(50) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] - -(51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) - -(52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] - -(53) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#34, ws_item_sk#35, ws_ext_sales_price#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_sold_date_sk#34, ws_item_sk#35, ws_ext_sales_price#36] - -(55) Filter [codegen id : 13] -Input [3]: [ws_sold_date_sk#34, ws_item_sk#35, ws_ext_sales_price#36] -Condition : (isnotnull(ws_item_sk#35) AND isnotnull(ws_sold_date_sk#34)) - -(56) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] - -(57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#4] -Join condition: None - -(58) Project [codegen id : 13] -Output [3]: [ws_sold_date_sk#34, ws_ext_sales_price#36, i_item_id#5] -Input [5]: [ws_sold_date_sk#34, ws_item_sk#35, ws_ext_sales_price#36, i_item_sk#4, i_item_id#5] - -(59) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(60) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#7, d_date#8] - -(61) Filter [codegen id : 12] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) - -(62) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(63) ColumnarToRow [codegen id : 11] -Input [2]: [d_date#8, d_week_seq#9] - -(64) Filter [codegen id : 11] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) - -(65) Project [codegen id : 11] -Output [1]: [d_date#8 AS d_date#8#37] -Input [2]: [d_date#8, d_week_seq#9] - -(66) BroadcastExchange -Input [1]: [d_date#8#37] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] - -(67) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#37] -Join condition: None - -(68) Project [codegen id : 12] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] - -(69) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] - -(70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#34] -Right keys [1]: [d_date_sk#7] -Join condition: None - -(71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#36, i_item_id#5] -Input [4]: [ws_sold_date_sk#34, ws_ext_sales_price#36, i_item_id#5, d_date_sk#7] - -(72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#36, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#36))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#5, sum#41] - -(73) Exchange -Input [2]: [i_item_id#5, sum#41] -Arguments: hashpartitioning(i_item_id#5, 5), true, [id=#42] - -(74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#5, sum#41] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#36))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#36))#43] -Results [2]: [i_item_id#5 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#36))#43,17,2) AS ws_item_rev#45] - -(75) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) - -(76) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] - -(77) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) - -(78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] - -(79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#10, [id=#11] -* Project (83) -+- * Filter (82) - +- * ColumnarToRow (81) - +- Scan parquet default.date_dim (80) - - -(80) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] -ReadSchema: struct - -(81) ColumnarToRow [codegen id : 1] -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)) - -(83) Project [codegen id : 1] -Output [1]: [d_week_seq#9] -Input [2]: [d_date#8, d_week_seq#9] - -Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] - -Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q58/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q58/simplified.txt deleted file mode 100644 index 5081efe94..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q58/simplified.txt +++ /dev/null @@ -1,125 +0,0 @@ -TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (15) - Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] - Project [item_id,ss_item_rev,cs_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [i_item_sk,i_item_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date] - Filter [d_week_seq] - Subquery #1 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (9) - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (8) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - Project [d_date] - Filter [d_week_seq] - ReusedSubquery [d_week_seq] #1 - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (14) - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - InputAdapter - Exchange [i_item_id] #10 - WholeStageCodegen (13) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (12) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (11) - Project [d_date] - Filter [d_week_seq] - ReusedSubquery [d_week_seq] #1 - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q59/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q59/explain.txt deleted file mode 100644 index 6edd0e4b0..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q59/explain.txt +++ /dev/null @@ -1,249 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * HashAggregate (12) - : : : +- Exchange (11) - : : : +- * HashAggregate (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.store (13) - : +- BroadcastExchange (23) - : +- * Project (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.date_dim (19) - +- BroadcastExchange (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * HashAggregate (27) - : : +- ReusedExchange (26) - : +- BroadcastExchange (31) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet default.store (28) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.date_dim (34) - - -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_sales_price#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_sales_price#3] - -(3) Filter [codegen id : 2] -Input [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_sales_price#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) - -(7) BroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] - -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(9) Project [codegen id : 2] -Output [4]: [ss_store_sk#2, ss_sales_price#3, d_week_seq#5, d_day_name#6] -Input [6]: [ss_sold_date_sk#1, ss_store_sk#2, ss_sales_price#3, d_date_sk#4, d_week_seq#5, d_day_name#6] - -(10) HashAggregate [codegen id : 2] -Input [4]: [ss_store_sk#2, ss_sales_price#3, d_week_seq#5, d_day_name#6] -Keys [2]: [d_week_seq#5, ss_store_sk#2] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday) THEN ss_sales_price#3 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday) THEN ss_sales_price#3 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday) THEN ss_sales_price#3 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#3 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday) THEN ss_sales_price#3 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday) THEN ss_sales_price#3 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday) THEN ss_sales_price#3 ELSE null END))] -Aggregate Attributes [7]: [sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Results [9]: [d_week_seq#5, ss_store_sk#2, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20, sum#21] - -(11) Exchange -Input [9]: [d_week_seq#5, ss_store_sk#2, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20, sum#21] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#2, 5), true, [id=#22] - -(12) HashAggregate [codegen id : 10] -Input [9]: [d_week_seq#5, ss_store_sk#2, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20, sum#21] -Keys [2]: [d_week_seq#5, ss_store_sk#2] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday) THEN ss_sales_price#3 ELSE null END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday) THEN ss_sales_price#3 ELSE null END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday) THEN ss_sales_price#3 ELSE null END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday) THEN ss_sales_price#3 ELSE null END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#3 ELSE null END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday) THEN ss_sales_price#3 ELSE null END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday) THEN ss_sales_price#3 ELSE null END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday) THEN ss_sales_price#3 ELSE null END))#29] -Results [9]: [d_week_seq#5, ss_store_sk#2, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday) THEN ss_sales_price#3 ELSE null END))#23,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday) THEN ss_sales_price#3 ELSE null END))#24,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday) THEN ss_sales_price#3 ELSE null END))#25,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#3 ELSE null END))#26,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday) THEN ss_sales_price#3 ELSE null END))#27,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday) THEN ss_sales_price#3 ELSE null END))#28,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday) THEN ss_sales_price#3 ELSE null END))#29,17,2) AS sat_sales#36] - -(13) Scan parquet default.store -Output [3]: [s_store_sk#37, s_store_id#38, s_store_name#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#37, s_store_id#38, s_store_name#39] - -(15) Filter [codegen id : 3] -Input [3]: [s_store_sk#37, s_store_id#38, s_store_name#39] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) - -(16) BroadcastExchange -Input [3]: [s_store_sk#37, s_store_id#38, s_store_name#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] - -(17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#37] -Join condition: None - -(18) Project [codegen id : 10] -Output [10]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#38, s_store_name#39] -Input [12]: [d_week_seq#5, ss_store_sk#2, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_sk#37, s_store_id#38, s_store_name#39] - -(19) Scan parquet default.date_dim -Output [2]: [d_month_seq#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 4] -Input [2]: [d_month_seq#41, d_week_seq#42] - -(21) Filter [codegen id : 4] -Input [2]: [d_month_seq#41, d_week_seq#42] -Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= 1212)) AND (d_month_seq#41 <= 1223)) AND isnotnull(d_week_seq#42)) - -(22) Project [codegen id : 4] -Output [1]: [d_week_seq#42] -Input [2]: [d_month_seq#41, d_week_seq#42] - -(23) BroadcastExchange -Input [1]: [d_week_seq#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] - -(24) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#42] -Join condition: None - -(25) Project [codegen id : 10] -Output [10]: [s_store_name#39 AS s_store_name1#44, d_week_seq#5 AS d_week_seq1#45, s_store_id#38 AS s_store_id1#46, sun_sales#30 AS sun_sales1#47, mon_sales#31 AS mon_sales1#48, tue_sales#32 AS tue_sales1#49, wed_sales#33 AS wed_sales1#50, thu_sales#34 AS thu_sales1#51, fri_sales#35 AS fri_sales1#52, sat_sales#36 AS sat_sales1#53] -Input [11]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#38, s_store_name#39, d_week_seq#42] - -(26) ReusedExchange [Reuses operator id: 11] -Output [9]: [d_week_seq#5, ss_store_sk#2, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] - -(27) HashAggregate [codegen id : 9] -Input [9]: [d_week_seq#5, ss_store_sk#2, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] -Keys [2]: [d_week_seq#5, ss_store_sk#2] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday) THEN ss_sales_price#3 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday) THEN ss_sales_price#3 ELSE null END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday) THEN ss_sales_price#3 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday) THEN ss_sales_price#3 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday) THEN ss_sales_price#3 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#3 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday) THEN ss_sales_price#3 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday) THEN ss_sales_price#3 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday) THEN ss_sales_price#3 ELSE null END))#67] -Results [9]: [d_week_seq#5, ss_store_sk#2, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday) THEN ss_sales_price#3 ELSE null END))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday) THEN ss_sales_price#3 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday) THEN ss_sales_price#3 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#3 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday) THEN ss_sales_price#3 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday) THEN ss_sales_price#3 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday) THEN ss_sales_price#3 ELSE null END))#67,17,2) AS sat_sales#36] - -(28) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] - -(30) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) - -(31) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#37] -Join condition: None - -(33) Project [codegen id : 9] -Output [9]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#38] -Input [11]: [d_week_seq#5, ss_store_sk#2, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_sk#37, s_store_id#38] - -(34) Scan parquet default.date_dim -Output [2]: [d_month_seq#69, d_week_seq#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 8] -Input [2]: [d_month_seq#69, d_week_seq#70] - -(36) Filter [codegen id : 8] -Input [2]: [d_month_seq#69, d_week_seq#70] -Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) - -(37) Project [codegen id : 8] -Output [1]: [d_week_seq#70] -Input [2]: [d_month_seq#69, d_week_seq#70] - -(38) BroadcastExchange -Input [1]: [d_week_seq#70] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#70] -Join condition: None - -(40) Project [codegen id : 9] -Output [9]: [d_week_seq#5 AS d_week_seq2#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_sales2#76, wed_sales#33 AS wed_sales2#77, thu_sales#34 AS thu_sales2#78, fri_sales#35 AS fri_sales2#79, sat_sales#36 AS sat_sales2#80] -Input [10]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#38, d_week_seq#70] - -(41) BroadcastExchange -Input [9]: [d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#81] - -(42) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#73, (d_week_seq2#72 - 52)] -Join condition: None - -(43) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#84, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#77)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#85, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#78)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#86, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#79)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#87, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#80)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#88] -Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] - -(44) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales2)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] -Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales2)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q59/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q59/simplified.txt deleted file mode 100644 index 3f3cc409e..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q59/simplified.txt +++ /dev/null @@ -1,66 +0,0 @@ -TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - WholeStageCodegen (10) - Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] - Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - InputAdapter - Exchange [d_week_seq,ss_store_sk] #1 - WholeStageCodegen (2) - HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [d_date_sk,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Filter [s_store_sk,s_store_id] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [d_week_seq] - Filter [d_month_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (9) - Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - InputAdapter - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - Filter [s_store_sk,s_store_id] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - Project [d_week_seq] - Filter [d_month_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_week_seq] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q6/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q6/explain.txt deleted file mode 100644 index 822d24b2f..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q6/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * Filter (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.customer_address (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.customer (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.store_sales (10) - : +- BroadcastExchange (20) - : +- * Project (19) - : +- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet default.date_dim (16) - +- BroadcastExchange (36) - +- * Project (35) - +- * Filter (34) - +- * BroadcastHashJoin LeftOuter BuildRight (33) - :- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.item (23) - +- BroadcastExchange (32) - +- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Filter (28) - +- * ColumnarToRow (27) - +- Scan parquet default.item (26) - - -(1) Scan parquet default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#1, ca_state#2] - -(3) Filter [codegen id : 7] -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(4) Scan parquet default.customer -Output [2]: [c_customer_sk#3, c_current_addr_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#3, c_current_addr_sk#4] - -(6) Filter [codegen id : 1] -Input [2]: [c_customer_sk#3, c_current_addr_sk#4] -Condition : (isnotnull(c_current_addr_sk#4) AND isnotnull(c_customer_sk#3)) - -(7) BroadcastExchange -Input [2]: [c_customer_sk#3, c_current_addr_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#5] - -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ca_address_sk#1] -Right keys [1]: [c_current_addr_sk#4] -Join condition: None - -(9) Project [codegen id : 7] -Output [2]: [ca_state#2, c_customer_sk#3] -Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] - -(10) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#6, ss_item_sk#7, ss_customer_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [ss_sold_date_sk#6, ss_item_sk#7, ss_customer_sk#8] - -(12) Filter [codegen id : 2] -Input [3]: [ss_sold_date_sk#6, ss_item_sk#7, ss_customer_sk#8] -Condition : ((isnotnull(ss_customer_sk#8) AND isnotnull(ss_sold_date_sk#6)) AND isnotnull(ss_item_sk#7)) - -(13) BroadcastExchange -Input [3]: [ss_sold_date_sk#6, ss_item_sk#7, ss_customer_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, false] as bigint)),false), [id=#9] - -(14) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#8] -Join condition: None - -(15) Project [codegen id : 7] -Output [3]: [ca_state#2, ss_sold_date_sk#6, ss_item_sk#7] -Input [5]: [ca_state#2, c_customer_sk#3, ss_sold_date_sk#6, ss_item_sk#7, ss_customer_sk#8] - -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#10, d_month_seq#11] - -(18) Filter [codegen id : 3] -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(19) Project [codegen id : 3] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_month_seq#11] - -(20) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] - -(21) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(22) Project [codegen id : 7] -Output [2]: [ca_state#2, ss_item_sk#7] -Input [4]: [ca_state#2, ss_sold_date_sk#6, ss_item_sk#7, d_date_sk#10] - -(23) Scan parquet default.item -Output [3]: [i_item_sk#15, i_current_price#16, i_category#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 6] -Input [3]: [i_item_sk#15, i_current_price#16, i_category#17] - -(25) Filter [codegen id : 6] -Input [3]: [i_item_sk#15, i_current_price#16, i_category#17] -Condition : (isnotnull(i_current_price#16) AND isnotnull(i_item_sk#15)) - -(26) Scan parquet default.item -Output [2]: [i_current_price#16, i_category#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category)] -ReadSchema: struct - -(27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#16, i_category#17] - -(28) Filter [codegen id : 4] -Input [2]: [i_current_price#16, i_category#17] -Condition : isnotnull(i_category#17) - -(29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#16, i_category#17] -Keys [1]: [i_category#17] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#16))] -Aggregate Attributes [2]: [sum#18, count#19] -Results [3]: [i_category#17, sum#20, count#21] - -(30) Exchange -Input [3]: [i_category#17, sum#20, count#21] -Arguments: hashpartitioning(i_category#17, 5), true, [id=#22] - -(31) HashAggregate [codegen id : 5] -Input [3]: [i_category#17, sum#20, count#21] -Keys [1]: [i_category#17] -Functions [1]: [avg(UnscaledValue(i_current_price#16))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#16))#23] -Results [2]: [cast((avg(UnscaledValue(i_current_price#16))#23 / 100.0) as decimal(11,6)) AS avg(i_current_price)#24, i_category#17 AS i_category#17#25] - -(32) BroadcastExchange -Input [2]: [avg(i_current_price)#24, i_category#17#25] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#26] - -(33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_category#17] -Right keys [1]: [i_category#17#25] -Join condition: None - -(34) Filter [codegen id : 6] -Input [5]: [i_item_sk#15, i_current_price#16, i_category#17, avg(i_current_price)#24, i_category#17#25] -Condition : (cast(i_current_price#16 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#24)), DecimalType(14,7), true)) - -(35) Project [codegen id : 6] -Output [1]: [i_item_sk#15] -Input [5]: [i_item_sk#15, i_current_price#16, i_category#17, avg(i_current_price)#24, i_category#17#25] - -(36) BroadcastExchange -Input [1]: [i_item_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] - -(37) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#15] -Join condition: None - -(38) Project [codegen id : 7] -Output [1]: [ca_state#2] -Input [3]: [ca_state#2, ss_item_sk#7, i_item_sk#15] - -(39) HashAggregate [codegen id : 7] -Input [1]: [ca_state#2] -Keys [1]: [ca_state#2] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [2]: [ca_state#2, count#29] - -(40) Exchange -Input [2]: [ca_state#2, count#29] -Arguments: hashpartitioning(ca_state#2, 5), true, [id=#30] - -(41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#29] -Keys [1]: [ca_state#2] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [3]: [ca_state#2 AS state#32, count(1)#31 AS cnt#33, count(1)#31 AS count(1)#34] - -(42) Filter [codegen id : 8] -Input [3]: [state#32, cnt#33, count(1)#34] -Condition : (count(1)#34 >= 10) - -(43) Project [codegen id : 8] -Output [2]: [state#32, cnt#33] -Input [3]: [state#32, cnt#33, count(1)#34] - -(44) TakeOrderedAndProject -Input [2]: [state#32, cnt#33] -Arguments: 100, [cnt#33 ASC NULLS FIRST], [state#32, cnt#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* HashAggregate (51) -+- Exchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * Filter (47) - +- * ColumnarToRow (46) - +- Scan parquet default.date_dim (45) - - -(45) Scan parquet default.date_dim -Output [3]: [d_month_seq#11, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#11, d_year#35, d_moy#36] - -(47) Filter [codegen id : 1] -Input [3]: [d_month_seq#11, d_year#35, d_moy#36] -Condition : (((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 1)) - -(48) Project [codegen id : 1] -Output [1]: [d_month_seq#11] -Input [3]: [d_month_seq#11, d_year#35, d_moy#36] - -(49) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#11] -Keys [1]: [d_month_seq#11] -Functions: [] -Aggregate Attributes: [] -Results [1]: [d_month_seq#11] - -(50) Exchange -Input [1]: [d_month_seq#11] -Arguments: hashpartitioning(d_month_seq#11, 5), true, [id=#37] - -(51) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#11] -Keys [1]: [d_month_seq#11] -Functions: [] -Aggregate Attributes: [] -Results [1]: [d_month_seq#11] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q6/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q6/simplified.txt deleted file mode 100644 index 97ba16294..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q6/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [cnt,state] - WholeStageCodegen (8) - Project [state,cnt] - Filter [count(1)] - HashAggregate [ca_state,count] [count(1),state,cnt,count(1),count] - InputAdapter - Exchange [ca_state] #1 - WholeStageCodegen (7) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_sold_date_sk,ss_item_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [ss_customer_sk,ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #1 - WholeStageCodegen (2) - HashAggregate [d_month_seq] - InputAdapter - Exchange [d_month_seq] #5 - WholeStageCodegen (1) - HashAggregate [d_month_seq] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [i_item_sk] - Filter [i_current_price,avg(i_current_price)] - BroadcastHashJoin [i_category,i_category] - Filter [i_current_price,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_category] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] - InputAdapter - Exchange [i_category] #8 - WholeStageCodegen (4) - HashAggregate [i_category,i_current_price] [sum,count,sum,count] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_current_price,i_category] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q60/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q60/explain.txt deleted file mode 100644 index f838f8f1a..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q60/explain.txt +++ /dev/null @@ -1,378 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) - :- * HashAggregate (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer_address (11) - : +- BroadcastExchange (27) - : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : :- * Filter (20) - : : +- * ColumnarToRow (19) - : : +- Scan parquet default.item (18) - : +- BroadcastExchange (25) - : +- * Project (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (41) - : : +- * BroadcastHashJoin Inner BuildRight (40) - : : :- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.catalog_sales (33) - : : : +- ReusedExchange (36) - : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) - - -(1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] - -(3) Filter [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_item_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_moy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -Condition : ((((isnotnull(d_year#6) AND isnotnull(d_moy#7)) AND (d_year#6 = 1998)) AND (d_moy#7 = 9)) AND isnotnull(d_date_sk#5)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] - -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(10) Project [codegen id : 5] -Output [3]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, d_date_sk#5] - -(11) Scan parquet default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] - -(13) Filter [codegen id : 2] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(14) Project [codegen id : 2] -Output [1]: [ca_address_sk#9] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] - -(15) BroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(17) Project [codegen id : 5] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#4] -Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] - -(18) Scan parquet default.item -Output [2]: [i_item_sk#12, i_item_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] - -(20) Filter [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] -Condition : isnotnull(i_item_sk#12) - -(21) Scan parquet default.item -Output [2]: [i_item_id#13, i_category#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music)] -ReadSchema: struct - -(22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#13, i_category#14] - -(23) Filter [codegen id : 3] -Input [2]: [i_item_id#13, i_category#14] -Condition : (isnotnull(i_category#14) AND (i_category#14 = Music)) - -(24) Project [codegen id : 3] -Output [1]: [i_item_id#13 AS i_item_id#13#15] -Input [2]: [i_item_id#13, i_category#14] - -(25) BroadcastExchange -Input [1]: [i_item_id#13#15] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#16] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_id#13] -Right keys [1]: [i_item_id#13#15] -Join condition: None - -(27) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] - -(28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(29) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#4, i_item_id#13] -Input [4]: [ss_item_sk#2, ss_ext_sales_price#4, i_item_sk#12, i_item_id#13] - -(30) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#4, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [1]: [sum#18] -Results [2]: [i_item_id#13, sum#19] - -(31) Exchange -Input [2]: [i_item_id#13, sum#19] -Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#20] - -(32) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#13, sum#19] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#4))#21] -Results [2]: [i_item_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] - -(33) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] - -(35) Filter [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Condition : ((isnotnull(cs_sold_date_sk#23) AND isnotnull(cs_bill_addr_sk#24)) AND isnotnull(cs_item_sk#25)) - -(36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(38) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_date_sk#5] - -(39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#9] - -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(41) Project [codegen id : 11] -Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#9] - -(42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#12, i_item_id#13] - -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#13] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#12, i_item_id#13] - -(45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#13, sum#28] - -(46) Exchange -Input [2]: [i_item_id#13, sum#28] -Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#29] - -(47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#13, sum#28] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#30] -Results [2]: [i_item_id#13, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] - -(48) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] - -(50) Filter [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Condition : ((isnotnull(ws_sold_date_sk#32) AND isnotnull(ws_bill_addr_sk#34)) AND isnotnull(ws_item_sk#33)) - -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, d_date_sk#5] - -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#9] - -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#9] - -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#12, i_item_id#13] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#12] -Join condition: None - -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#13] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#12, i_item_id#13] - -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#36] -Results [2]: [i_item_id#13, sum#37] - -(61) Exchange -Input [2]: [i_item_id#13, sum#37] -Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#38] - -(62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#13, sum#37] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#39] -Results [2]: [i_item_id#13, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] - -(63) Union - -(64) HashAggregate [codegen id : 19] -Input [2]: [i_item_id#13, total_sales#22] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(total_sales#22)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_item_id#13, sum#43, isEmpty#44] - -(65) Exchange -Input [3]: [i_item_id#13, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#45] - -(66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#13, sum#43, isEmpty#44] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(total_sales#22)] -Aggregate Attributes [1]: [sum(total_sales#22)#46] -Results [2]: [i_item_id#13, sum(total_sales#22)#46 AS total_sales#47] - -(67) TakeOrderedAndProject -Input [2]: [i_item_id#13, total_sales#47] -Arguments: 100, [i_item_id#13 ASC NULLS FIRST, total_sales#47 ASC NULLS FIRST], [i_item_id#13, total_sales#47] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q60/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q60/simplified.txt deleted file mode 100644 index fb9e4e507..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q60/simplified.txt +++ /dev/null @@ -1,101 +0,0 @@ -TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (20) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [i_item_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_id,i_category] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q61/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q61/explain.txt deleted file mode 100644 index f56f48726..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q61/explain.txt +++ /dev/null @@ -1,396 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- BroadcastNestedLoopJoin Inner BuildRight (70) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (30) - : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : :- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * Project (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet default.store (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * Project (14) - : : : : : +- * Filter (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- Scan parquet default.promotion (11) - : : : : +- BroadcastExchange (22) - : : : : +- * Project (21) - : : : : +- * Filter (20) - : : : : +- * ColumnarToRow (19) - : : : : +- Scan parquet default.date_dim (18) - : : : +- BroadcastExchange (28) - : : : +- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet default.customer (25) - : : +- BroadcastExchange (35) - : : +- * Project (34) - : : +- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet default.customer_address (31) - : +- BroadcastExchange (42) - : +- * Project (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.item (38) - +- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (56) - : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : :- * Project (53) - : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : :- * Filter (50) - : : : : : +- * ColumnarToRow (49) - : : : : : +- Scan parquet default.store_sales (48) - : : : : +- ReusedExchange (51) - : : : +- ReusedExchange (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (60) - +- ReusedExchange (63) - - -(1) Scan parquet default.store_sales -Output [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_promo_sk#5, ss_ext_sales_price#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_promo_sk#5, ss_ext_sales_price#6] - -(3) Filter [codegen id : 7] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_promo_sk#5, ss_ext_sales_price#6] -Condition : ((((isnotnull(ss_store_sk#4) AND isnotnull(ss_promo_sk#5)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_item_sk#2)) - -(4) Scan parquet default.store -Output [2]: [s_store_sk#7, s_gmt_offset#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [s_store_sk#7, s_gmt_offset#8] - -(6) Filter [codegen id : 1] -Input [2]: [s_store_sk#7, s_gmt_offset#8] -Condition : ((isnotnull(s_gmt_offset#8) AND (s_gmt_offset#8 = -5.00)) AND isnotnull(s_store_sk#7)) - -(7) Project [codegen id : 1] -Output [1]: [s_store_sk#7] -Input [2]: [s_store_sk#7, s_gmt_offset#8] - -(8) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#7] -Join condition: None - -(10) Project [codegen id : 7] -Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_promo_sk#5, ss_ext_sales_price#6] -Input [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_promo_sk#5, ss_ext_sales_price#6, s_store_sk#7] - -(11) Scan parquet default.promotion -Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [Or(Or(EqualTo(p_channel_dmail,Y),EqualTo(p_channel_email,Y)),EqualTo(p_channel_tv,Y)), IsNotNull(p_promo_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] - -(13) Filter [codegen id : 2] -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Condition : ((((p_channel_dmail#11 = Y) OR (p_channel_email#12 = Y)) OR (p_channel_tv#13 = Y)) AND isnotnull(p_promo_sk#10)) - -(14) Project [codegen id : 2] -Output [1]: [p_promo_sk#10] -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] - -(15) BroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_promo_sk#5] -Right keys [1]: [p_promo_sk#10] -Join condition: None - -(17) Project [codegen id : 7] -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ext_sales_price#6] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_promo_sk#5, ss_ext_sales_price#6, p_promo_sk#10] - -(18) Scan parquet default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#15, d_year#16, d_moy#17] - -(20) Filter [codegen id : 3] -Input [3]: [d_date_sk#15, d_year#16, d_moy#17] -Condition : ((((isnotnull(d_year#16) AND isnotnull(d_moy#17)) AND (d_year#16 = 1998)) AND (d_moy#17 = 11)) AND isnotnull(d_date_sk#15)) - -(21) Project [codegen id : 3] -Output [1]: [d_date_sk#15] -Input [3]: [d_date_sk#15, d_year#16, d_moy#17] - -(22) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] - -(23) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] -Join condition: None - -(24) Project [codegen id : 7] -Output [3]: [ss_item_sk#2, ss_customer_sk#3, ss_ext_sales_price#6] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ext_sales_price#6, d_date_sk#15] - -(25) Scan parquet default.customer -Output [2]: [c_customer_sk#19, c_current_addr_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#19, c_current_addr_sk#20] - -(27) Filter [codegen id : 4] -Input [2]: [c_customer_sk#19, c_current_addr_sk#20] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_current_addr_sk#20)) - -(28) BroadcastExchange -Input [2]: [c_customer_sk#19, c_current_addr_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] - -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_customer_sk#3] -Right keys [1]: [c_customer_sk#19] -Join condition: None - -(30) Project [codegen id : 7] -Output [3]: [ss_item_sk#2, ss_ext_sales_price#6, c_current_addr_sk#20] -Input [5]: [ss_item_sk#2, ss_customer_sk#3, ss_ext_sales_price#6, c_customer_sk#19, c_current_addr_sk#20] - -(31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_gmt_offset#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(32) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#22, ca_gmt_offset#23] - -(33) Filter [codegen id : 5] -Input [2]: [ca_address_sk#22, ca_gmt_offset#23] -Condition : ((isnotnull(ca_gmt_offset#23) AND (ca_gmt_offset#23 = -5.00)) AND isnotnull(ca_address_sk#22)) - -(34) Project [codegen id : 5] -Output [1]: [ca_address_sk#22] -Input [2]: [ca_address_sk#22, ca_gmt_offset#23] - -(35) BroadcastExchange -Input [1]: [ca_address_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] - -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#20] -Right keys [1]: [ca_address_sk#22] -Join condition: None - -(37) Project [codegen id : 7] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#6] -Input [4]: [ss_item_sk#2, ss_ext_sales_price#6, c_current_addr_sk#20, ca_address_sk#22] - -(38) Scan parquet default.item -Output [2]: [i_item_sk#25, i_category#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Jewelry), IsNotNull(i_item_sk)] -ReadSchema: struct - -(39) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#25, i_category#26] - -(40) Filter [codegen id : 6] -Input [2]: [i_item_sk#25, i_category#26] -Condition : ((isnotnull(i_category#26) AND (i_category#26 = Jewelry)) AND isnotnull(i_item_sk#25)) - -(41) Project [codegen id : 6] -Output [1]: [i_item_sk#25] -Input [2]: [i_item_sk#25, i_category#26] - -(42) BroadcastExchange -Input [1]: [i_item_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] - -(43) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#25] -Join condition: None - -(44) Project [codegen id : 7] -Output [1]: [ss_ext_sales_price#6] -Input [3]: [ss_item_sk#2, ss_ext_sales_price#6, i_item_sk#25] - -(45) HashAggregate [codegen id : 7] -Input [1]: [ss_ext_sales_price#6] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum#28] -Results [1]: [sum#29] - -(46) Exchange -Input [1]: [sum#29] -Arguments: SinglePartition, true, [id=#30] - -(47) HashAggregate [codegen id : 8] -Input [1]: [sum#29] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#31] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#31,17,2) AS promotions#32] - -(48) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 14] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] - -(50) Filter [codegen id : 14] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_item_sk#2)) - -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#7] - -(52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#7] -Join condition: None - -(53) Project [codegen id : 14] -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ext_sales_price#6] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6, s_store_sk#7] - -(54) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#15] - -(55) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] -Join condition: None - -(56) Project [codegen id : 14] -Output [3]: [ss_item_sk#2, ss_customer_sk#3, ss_ext_sales_price#6] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ext_sales_price#6, d_date_sk#15] - -(57) ReusedExchange [Reuses operator id: 28] -Output [2]: [c_customer_sk#19, c_current_addr_sk#20] - -(58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#3] -Right keys [1]: [c_customer_sk#19] -Join condition: None - -(59) Project [codegen id : 14] -Output [3]: [ss_item_sk#2, ss_ext_sales_price#6, c_current_addr_sk#20] -Input [5]: [ss_item_sk#2, ss_customer_sk#3, ss_ext_sales_price#6, c_customer_sk#19, c_current_addr_sk#20] - -(60) ReusedExchange [Reuses operator id: 35] -Output [1]: [ca_address_sk#22] - -(61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_current_addr_sk#20] -Right keys [1]: [ca_address_sk#22] -Join condition: None - -(62) Project [codegen id : 14] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#6] -Input [4]: [ss_item_sk#2, ss_ext_sales_price#6, c_current_addr_sk#20, ca_address_sk#22] - -(63) ReusedExchange [Reuses operator id: 42] -Output [1]: [i_item_sk#25] - -(64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#25] -Join condition: None - -(65) Project [codegen id : 14] -Output [1]: [ss_ext_sales_price#6] -Input [3]: [ss_item_sk#2, ss_ext_sales_price#6, i_item_sk#25] - -(66) HashAggregate [codegen id : 14] -Input [1]: [ss_ext_sales_price#6] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum#33] -Results [1]: [sum#34] - -(67) Exchange -Input [1]: [sum#34] -Arguments: SinglePartition, true, [id=#35] - -(68) HashAggregate [codegen id : 15] -Input [1]: [sum#34] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#36] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#36,17,2) AS total#37] - -(69) BroadcastExchange -Input [1]: [total#37] -Arguments: IdentityBroadcastMode, [id=#38] - -(70) BroadcastNestedLoopJoin -Join condition: None - -(71) Project [codegen id : 16] -Output [3]: [promotions#32, total#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#32 as decimal(15,4))) / promote_precision(cast(total#37 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#39] -Input [2]: [promotions#32, total#37] - -(72) TakeOrderedAndProject -Input [3]: [promotions#32, total#37, (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#39] -Arguments: 100, [promotions#32 ASC NULLS FIRST, total#37 ASC NULLS FIRST], [promotions#32, total#37, (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#39] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q61/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q61/simplified.txt deleted file mode 100644 index da7565167..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q61/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject [promotions,total,(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))] - WholeStageCodegen (16) - Project [promotions,total] - InputAdapter - BroadcastNestedLoopJoin - WholeStageCodegen (8) - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - InputAdapter - Exchange #1 - WholeStageCodegen (7) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_promo_sk,ss_sold_date_sk,ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [s_store_sk] - Filter [s_gmt_offset,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_gmt_offset] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [p_promo_sk] - Filter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [i_item_sk] - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_category] - BroadcastExchange #8 - WholeStageCodegen (15) - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - InputAdapter - Exchange #9 - WholeStageCodegen (14) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_sold_date_sk,ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - InputAdapter - ReusedExchange [s_store_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #6 - InputAdapter - ReusedExchange [i_item_sk] #7 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q62/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q62/explain.txt deleted file mode 100644 index 05ce467c3..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q62/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.web_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.warehouse (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.ship_mode (10) - : +- BroadcastExchange (19) - : +- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet default.web_site (16) - +- BroadcastExchange (26) - +- * Project (25) - +- * Filter (24) - +- * ColumnarToRow (23) - +- Scan parquet default.date_dim (22) - - -(1) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] - -(3) Filter [codegen id : 5] -Input [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] -Condition : (((isnotnull(ws_warehouse_sk#5) AND isnotnull(ws_ship_mode_sk#4)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_ship_date_sk#2)) - -(4) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) Filter [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(7) BroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#5] -Right keys [1]: [w_warehouse_sk#6] -Join condition: None - -(9) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, w_warehouse_name#7] -Input [7]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5, w_warehouse_sk#6, w_warehouse_name#7] - -(10) Scan parquet default.ship_mode -Output [2]: [sm_ship_mode_sk#9, sm_type#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [sm_ship_mode_sk#9, sm_type#10] - -(12) Filter [codegen id : 2] -Input [2]: [sm_ship_mode_sk#9, sm_type#10] -Condition : isnotnull(sm_ship_mode_sk#9) - -(13) BroadcastExchange -Input [2]: [sm_ship_mode_sk#9, sm_type#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] - -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#4] -Right keys [1]: [sm_ship_mode_sk#9] -Join condition: None - -(15) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, w_warehouse_name#7, sm_type#10] -Input [7]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, w_warehouse_name#7, sm_ship_mode_sk#9, sm_type#10] - -(16) Scan parquet default.web_site -Output [2]: [web_site_sk#12, web_name#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [2]: [web_site_sk#12, web_name#13] - -(18) Filter [codegen id : 3] -Input [2]: [web_site_sk#12, web_name#13] -Condition : isnotnull(web_site_sk#12) - -(19) BroadcastExchange -Input [2]: [web_site_sk#12, web_name#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#12] -Join condition: None - -(21) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, w_warehouse_name#7, sm_type#10, web_name#13] -Input [7]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, w_warehouse_name#7, sm_type#10, web_site_sk#12, web_name#13] - -(22) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_month_seq#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(23) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#15, d_month_seq#16] - -(24) Filter [codegen id : 4] -Input [2]: [d_date_sk#15, d_month_seq#16] -Condition : (((isnotnull(d_month_seq#16) AND (d_month_seq#16 >= 1200)) AND (d_month_seq#16 <= 1211)) AND isnotnull(d_date_sk#15)) - -(25) Project [codegen id : 4] -Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_month_seq#16] - -(26) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] - -(27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_date_sk#2] -Right keys [1]: [d_date_sk#15] -Join condition: None - -(28) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, w_warehouse_name#7, sm_type#10, web_name#13] -Input [6]: [ws_sold_date_sk#1, ws_ship_date_sk#2, w_warehouse_name#7, sm_type#10, web_name#13, d_date_sk#15] - -(29) HashAggregate [codegen id : 5] -Input [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, w_warehouse_name#7, sm_type#10, web_name#13] -Keys [3]: [substr(w_warehouse_name#7, 1, 20) AS substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, web_name#13] -Functions [5]: [partial_sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum#19, sum#20, sum#21, sum#22, sum#23] -Results [8]: [substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, web_name#13, sum#24, sum#25, sum#26, sum#27, sum#28] - -(30) Exchange -Input [8]: [substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, web_name#13, sum#24, sum#25, sum#26, sum#27, sum#28] -Arguments: hashpartitioning(substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, web_name#13, 5), true, [id=#29] - -(31) HashAggregate [codegen id : 6] -Input [8]: [substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, web_name#13, sum#24, sum#25, sum#26, sum#27, sum#28] -Keys [3]: [substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, web_name#13] -Functions [5]: [sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33, sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34] -Results [8]: [substr(w_warehouse_name#7, 1, 20)#18 AS substr(w_warehouse_name, 1, 20)#35, sm_type#10, web_name#13, sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30 AS 30 days #36, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31 AS 31 - 60 days #37, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32 AS 61 - 90 days #38, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33 AS 91 - 120 days #39, sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34 AS >120 days #40] - -(32) TakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#10, web_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] -Arguments: 100, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, web_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q62/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q62/simplified.txt deleted file mode 100644 index 803326b2a..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q62/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) - HashAggregate [substr(w_warehouse_name, 1, 20),sm_type,web_name,sum,sum,sum,sum,sum] [sum(cast(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END as bigint)),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - InputAdapter - Exchange [substr(w_warehouse_name, 1, 20),sm_type,web_name] #1 - WholeStageCodegen (5) - HashAggregate [w_warehouse_name,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum] - Project [ws_sold_date_sk,ws_ship_date_sk,w_warehouse_name,sm_type,web_name] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_ship_date_sk,w_warehouse_name,sm_type,web_name] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,w_warehouse_name,sm_type] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,w_warehouse_name] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - Filter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [sm_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet default.ship_mode [sm_ship_mode_sk,sm_type] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_site [web_site_sk,web_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q63/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q63/explain.txt deleted file mode 100644 index 284a9203a..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q63/explain.txt +++ /dev/null @@ -1,180 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * Filter (30) - +- Window (29) - +- * Sort (28) - +- Exchange (27) - +- * HashAggregate (26) - +- Exchange (25) - +- * HashAggregate (24) - +- * Project (23) - +- * BroadcastHashJoin Inner BuildRight (22) - :- * Project (17) - : +- * BroadcastHashJoin Inner BuildRight (16) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet default.store_sales (5) - : +- BroadcastExchange (15) - : +- * Project (14) - : +- * Filter (13) - : +- * ColumnarToRow (12) - : +- Scan parquet default.date_dim (11) - +- BroadcastExchange (21) - +- * Filter (20) - +- * ColumnarToRow (19) - +- Scan parquet default.store (18) - - -(1) Scan parquet default.item -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)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -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)) - -(4) Project [codegen id : 4] -Output [2]: [i_item_sk#1, i_manager_id#5] -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] - -(5) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#10, ss_item_sk#11, ss_store_sk#12, ss_sales_price#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_sold_date_sk#10, ss_item_sk#11, ss_store_sk#12, ss_sales_price#13] - -(7) Filter [codegen id : 1] -Input [4]: [ss_sold_date_sk#10, ss_item_sk#11, ss_store_sk#12, ss_sales_price#13] -Condition : ((isnotnull(ss_item_sk#11) AND isnotnull(ss_sold_date_sk#10)) AND isnotnull(ss_store_sk#12)) - -(8) BroadcastExchange -Input [4]: [ss_sold_date_sk#10, ss_item_sk#11, ss_store_sk#12, ss_sales_price#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#14] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#11] -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_sold_date_sk#10, ss_store_sk#12, ss_sales_price#13] -Input [6]: [i_item_sk#1, i_manager_id#5, ss_sold_date_sk#10, ss_item_sk#11, ss_store_sk#12, ss_sales_price#13] - -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1211,1205,1201,1206,1210,1207,1202,1209,1203,1208,1204]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] - -(13) Filter [codegen id : 2] -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET (1200,1211,1205,1201,1206,1210,1207,1202,1209,1203,1208,1204) AND isnotnull(d_date_sk#15)) - -(14) Project [codegen id : 2] -Output [2]: [d_date_sk#15, d_moy#17] -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] - -(15) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#15] -Join condition: None - -(17) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#12, ss_sales_price#13, d_moy#17] -Input [6]: [i_manager_id#5, ss_sold_date_sk#10, ss_store_sk#12, ss_sales_price#13, d_date_sk#15, d_moy#17] - -(18) Scan parquet default.store -Output [1]: [s_store_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#19] - -(20) Filter [codegen id : 3] -Input [1]: [s_store_sk#19] -Condition : isnotnull(s_store_sk#19) - -(21) BroadcastExchange -Input [1]: [s_store_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#12] -Right keys [1]: [s_store_sk#19] -Join condition: None - -(23) Project [codegen id : 4] -Output [3]: [i_manager_id#5, ss_sales_price#13, d_moy#17] -Input [5]: [i_manager_id#5, ss_store_sk#12, ss_sales_price#13, d_moy#17, s_store_sk#19] - -(24) HashAggregate [codegen id : 4] -Input [3]: [i_manager_id#5, ss_sales_price#13, d_moy#17] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [i_manager_id#5, d_moy#17, sum#22] - -(25) Exchange -Input [3]: [i_manager_id#5, d_moy#17, sum#22] -Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), true, [id=#23] - -(26) HashAggregate [codegen id : 5] -Input [3]: [i_manager_id#5, d_moy#17, sum#22] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#13))#24] -Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#13))#24,17,2) AS sum_sales#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#13))#24,17,2) AS _w0#26] - -(27) Exchange -Input [3]: [i_manager_id#5, sum_sales#25, _w0#26] -Arguments: hashpartitioning(i_manager_id#5, 5), true, [id=#27] - -(28) Sort [codegen id : 6] -Input [3]: [i_manager_id#5, sum_sales#25, _w0#26] -Arguments: [i_manager_id#5 ASC NULLS FIRST], false, 0 - -(29) Window -Input [3]: [i_manager_id#5, sum_sales#25, _w0#26] -Arguments: [avg(_w0#26) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#28], [i_manager_id#5] - -(30) Filter [codegen id : 7] -Input [4]: [i_manager_id#5, sum_sales#25, _w0#26, avg_monthly_sales#28] -Condition : (CASE WHEN (avg_monthly_sales#28 > 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#25 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#28 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#28 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000) - -(31) Project [codegen id : 7] -Output [3]: [i_manager_id#5, sum_sales#25, avg_monthly_sales#28] -Input [4]: [i_manager_id#5, sum_sales#25, _w0#26, avg_monthly_sales#28] - -(32) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#25, avg_monthly_sales#28] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#28 ASC NULLS FIRST, sum_sales#25 ASC NULLS FIRST], [i_manager_id#5, sum_sales#25, avg_monthly_sales#28] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q63/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q63/simplified.txt deleted file mode 100644 index 7272c01ab..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q63/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (7) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (6) - Sort [i_manager_id] - InputAdapter - Exchange [i_manager_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_manager_id,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_sold_date_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [i_item_sk,i_manager_id] - Filter [i_category,i_class,i_brand,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date_sk,d_moy] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q64/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q64/explain.txt deleted file mode 100644 index 0e658ff99..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q64/explain.txt +++ /dev/null @@ -1,918 +0,0 @@ -== Physical Plan == -* Sort (170) -+- Exchange (169) - +- * Project (168) - +- * BroadcastHashJoin Inner BuildRight (167) - :- * HashAggregate (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- * Project (102) - : +- * BroadcastHashJoin Inner BuildRight (101) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * Project (86) - : : : : +- * BroadcastHashJoin Inner BuildRight (85) - : : : : :- * Project (83) - : : : : : +- * BroadcastHashJoin Inner BuildRight (82) - : : : : : :- * Project (77) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : : : : :- * Project (74) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : : : : : :- * Project (68) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : : : : : : :- * Project (62) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (61) - : : : : : : : : : :- * Project (59) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : : : : : : : : :- * Project (53) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : : : : : : : : :- * Project (50) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : : : : : : : : :- * Project (44) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : : : : : : : : : : : :- * Project (38) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : : : : : : : : : : : :- * Project (32) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : : : : : : : : : : : :- * Project (26) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : : : : : : : : : : : : : : :- * Project (9) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : : : : : : : : : :- * Filter (3) - : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : : : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : : : : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : : : : : : : : : +- * Filter (6) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (5) - : : : : : : : : : : : : : : : : : +- Scan parquet default.store_returns (4) - : : : : : : : : : : : : : : : : +- BroadcastExchange (24) - : : : : : : : : : : : : : : : : +- * Project (23) - : : : : : : : : : : : : : : : : +- * Filter (22) - : : : : : : : : : : : : : : : : +- * HashAggregate (21) - : : : : : : : : : : : : : : : : +- Exchange (20) - : : : : : : : : : : : : : : : : +- * HashAggregate (19) - : : : : : : : : : : : : : : : : +- * Project (18) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : : : : : : : : : : : :- * Filter (12) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (11) - : : : : : : : : : : : : : : : : : +- Scan parquet default.catalog_sales (10) - : : : : : : : : : : : : : : : : +- BroadcastExchange (16) - : : : : : : : : : : : : : : : : +- * Filter (15) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : : : : : : : : : +- Scan parquet default.catalog_returns (13) - : : : : : : : : : : : : : : : +- BroadcastExchange (30) - : : : : : : : : : : : : : : : +- * Filter (29) - : : : : : : : : : : : : : : : +- * ColumnarToRow (28) - : : : : : : : : : : : : : : : +- Scan parquet default.date_dim (27) - : : : : : : : : : : : : : : +- BroadcastExchange (36) - : : : : : : : : : : : : : : +- * Filter (35) - : : : : : : : : : : : : : : +- * ColumnarToRow (34) - : : : : : : : : : : : : : : +- Scan parquet default.store (33) - : : : : : : : : : : : : : +- BroadcastExchange (42) - : : : : : : : : : : : : : +- * Filter (41) - : : : : : : : : : : : : : +- * ColumnarToRow (40) - : : : : : : : : : : : : : +- Scan parquet default.customer (39) - : : : : : : : : : : : : +- BroadcastExchange (48) - : : : : : : : : : : : : +- * Filter (47) - : : : : : : : : : : : : +- * ColumnarToRow (46) - : : : : : : : : : : : : +- Scan parquet default.date_dim (45) - : : : : : : : : : : : +- ReusedExchange (51) - : : : : : : : : : : +- BroadcastExchange (57) - : : : : : : : : : : +- * Filter (56) - : : : : : : : : : : +- * ColumnarToRow (55) - : : : : : : : : : : +- Scan parquet default.customer_demographics (54) - : : : : : : : : : +- ReusedExchange (60) - : : : : : : : : +- BroadcastExchange (66) - : : : : : : : : +- * Filter (65) - : : : : : : : : +- * ColumnarToRow (64) - : : : : : : : : +- Scan parquet default.promotion (63) - : : : : : : : +- BroadcastExchange (72) - : : : : : : : +- * Filter (71) - : : : : : : : +- * ColumnarToRow (70) - : : : : : : : +- Scan parquet default.household_demographics (69) - : : : : : : +- ReusedExchange (75) - : : : : : +- BroadcastExchange (81) - : : : : : +- * Filter (80) - : : : : : +- * ColumnarToRow (79) - : : : : : +- Scan parquet default.customer_address (78) - : : : : +- ReusedExchange (84) - : : : +- BroadcastExchange (90) - : : : +- * Filter (89) - : : : +- * ColumnarToRow (88) - : : : +- Scan parquet default.income_band (87) - : : +- ReusedExchange (93) - : +- BroadcastExchange (100) - : +- * Project (99) - : +- * Filter (98) - : +- * ColumnarToRow (97) - : +- Scan parquet default.item (96) - +- BroadcastExchange (166) - +- * HashAggregate (165) - +- Exchange (164) - +- * HashAggregate (163) - +- * Project (162) - +- * BroadcastHashJoin Inner BuildRight (161) - :- * Project (159) - : +- * BroadcastHashJoin Inner BuildRight (158) - : :- * Project (156) - : : +- * BroadcastHashJoin Inner BuildRight (155) - : : :- * Project (153) - : : : +- * BroadcastHashJoin Inner BuildRight (152) - : : : :- * Project (150) - : : : : +- * BroadcastHashJoin Inner BuildRight (149) - : : : : :- * Project (147) - : : : : : +- * BroadcastHashJoin Inner BuildRight (146) - : : : : : :- * Project (144) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (143) - : : : : : : :- * Project (141) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140) - : : : : : : : :- * Project (138) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137) - : : : : : : : : :- * Project (135) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134) - : : : : : : : : : :- * Project (132) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131) - : : : : : : : : : : :- * Project (129) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (128) - : : : : : : : : : : : :- * Project (126) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (125) - : : : : : : : : : : : : :- * Project (123) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (122) - : : : : : : : : : : : : : :- * Project (120) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (119) - : : : : : : : : : : : : : : :- * Project (114) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (113) - : : : : : : : : : : : : : : : :- * Project (111) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (110) - : : : : : : : : : : : : : : : : :- * Filter (108) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) - : : : : : : : : : : : : : : : : : +- Scan parquet default.store_sales (106) - : : : : : : : : : : : : : : : : +- ReusedExchange (109) - : : : : : : : : : : : : : : : +- ReusedExchange (112) - : : : : : : : : : : : : : : +- BroadcastExchange (118) - : : : : : : : : : : : : : : +- * Filter (117) - : : : : : : : : : : : : : : +- * ColumnarToRow (116) - : : : : : : : : : : : : : : +- Scan parquet default.date_dim (115) - : : : : : : : : : : : : : +- ReusedExchange (121) - : : : : : : : : : : : : +- ReusedExchange (124) - : : : : : : : : : : : +- ReusedExchange (127) - : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : +- ReusedExchange (136) - : : : : : : : +- ReusedExchange (139) - : : : : : : +- ReusedExchange (142) - : : : : : +- ReusedExchange (145) - : : : : +- ReusedExchange (148) - : : : +- ReusedExchange (151) - : : +- ReusedExchange (154) - : +- ReusedExchange (157) - +- ReusedExchange (160) - - -(1) Scan parquet default.store_sales -Output [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_ticket_number#9, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 20] -Input [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_ticket_number#9, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12] - -(3) Filter [codegen id : 20] -Input [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_ticket_number#9, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12] -Condition : ((((((((isnotnull(ss_item_sk#2) AND isnotnull(ss_ticket_number#9)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#7)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_cdemo_sk#4)) AND isnotnull(ss_promo_sk#8)) AND isnotnull(ss_hdemo_sk#5)) AND isnotnull(ss_addr_sk#6)) - -(4) Scan parquet default.store_returns -Output [2]: [sr_item_sk#13, sr_ticket_number#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [sr_item_sk#13, sr_ticket_number#14] - -(6) Filter [codegen id : 1] -Input [2]: [sr_item_sk#13, sr_ticket_number#14] -Condition : (isnotnull(sr_item_sk#13) AND isnotnull(sr_ticket_number#14)) - -(7) BroadcastExchange -Input [2]: [sr_item_sk#13, sr_ticket_number#14] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[1, bigint, false]),false), [id=#15] - -(8) BroadcastHashJoin [codegen id : 20] -Left keys [2]: [cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#9 as bigint)] -Right keys [2]: [sr_item_sk#13, sr_ticket_number#14] -Join condition: None - -(9) Project [codegen id : 20] -Output [11]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12] -Input [14]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_ticket_number#9, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, sr_item_sk#13, sr_ticket_number#14] - -(10) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#16, cs_order_number#17, cs_ext_list_price#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#16, cs_order_number#17, cs_ext_list_price#18] - -(12) Filter [codegen id : 3] -Input [3]: [cs_item_sk#16, cs_order_number#17, cs_ext_list_price#18] -Condition : (isnotnull(cs_item_sk#16) AND isnotnull(cs_order_number#17)) - -(13) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#19, cr_order_number#20, cr_refunded_cash#21, cr_reversed_charge#22, cr_store_credit#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 2] -Input [5]: [cr_item_sk#19, cr_order_number#20, cr_refunded_cash#21, cr_reversed_charge#22, cr_store_credit#23] - -(15) Filter [codegen id : 2] -Input [5]: [cr_item_sk#19, cr_order_number#20, cr_refunded_cash#21, cr_reversed_charge#22, cr_store_credit#23] -Condition : (isnotnull(cr_item_sk#19) AND isnotnull(cr_order_number#20)) - -(16) BroadcastExchange -Input [5]: [cr_item_sk#19, cr_order_number#20, cr_refunded_cash#21, cr_reversed_charge#22, cr_store_credit#23] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [id=#24] - -(17) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [cs_item_sk#16, cs_order_number#17] -Right keys [2]: [cr_item_sk#19, cr_order_number#20] -Join condition: None - -(18) Project [codegen id : 3] -Output [5]: [cs_item_sk#16, cs_ext_list_price#18, cr_refunded_cash#21, cr_reversed_charge#22, cr_store_credit#23] -Input [8]: [cs_item_sk#16, cs_order_number#17, cs_ext_list_price#18, cr_item_sk#19, cr_order_number#20, cr_refunded_cash#21, cr_reversed_charge#22, cr_store_credit#23] - -(19) HashAggregate [codegen id : 3] -Input [5]: [cs_item_sk#16, cs_ext_list_price#18, cr_refunded_cash#21, cr_reversed_charge#22, cr_store_credit#23] -Keys [1]: [cs_item_sk#16] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#18)), partial_sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#21 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#22 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#23 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [3]: [sum#25, sum#26, isEmpty#27] -Results [4]: [cs_item_sk#16, sum#28, sum#29, isEmpty#30] - -(20) Exchange -Input [4]: [cs_item_sk#16, sum#28, sum#29, isEmpty#30] -Arguments: hashpartitioning(cs_item_sk#16, 5), true, [id=#31] - -(21) HashAggregate [codegen id : 4] -Input [4]: [cs_item_sk#16, sum#28, sum#29, isEmpty#30] -Keys [1]: [cs_item_sk#16] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#18)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#21 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#22 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#23 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#18))#32, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#21 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#22 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#23 as decimal(9,2)))), DecimalType(9,2), true))#33] -Results [3]: [cs_item_sk#16, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#18))#32,17,2) AS sum(cs_ext_list_price#18)#34, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#21 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#22 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#23 as decimal(9,2)))), DecimalType(9,2), true))#33 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#21 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#22 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#23 as decimal(9,2)))), DecimalType(9,2), true))#35] - -(22) Filter [codegen id : 4] -Input [3]: [cs_item_sk#16, sum(cs_ext_list_price#18)#34, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#21 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#22 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#23 as decimal(9,2)))), DecimalType(9,2), true))#35] -Condition : (isnotnull(sum(cs_ext_list_price#18)#34) AND (cast(sum(cs_ext_list_price#18)#34 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#21 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#22 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#23 as decimal(9,2)))), DecimalType(9,2), true))#35)), DecimalType(21,2), true))) - -(23) Project [codegen id : 4] -Output [1]: [cs_item_sk#16] -Input [3]: [cs_item_sk#16, sum(cs_ext_list_price#18)#34, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#21 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#22 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#23 as decimal(9,2)))), DecimalType(9,2), true))#35] - -(24) BroadcastExchange -Input [1]: [cs_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] - -(25) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [cs_item_sk#16] -Join condition: None - -(26) Project [codegen id : 20] -Output [11]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12] -Input [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, cs_item_sk#16] - -(27) Scan parquet default.date_dim -Output [2]: [d_date_sk#37, d_year#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#37, d_year#38] - -(29) Filter [codegen id : 5] -Input [2]: [d_date_sk#37, d_year#38] -Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) - -(30) BroadcastExchange -Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] - -(31) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#37] -Join condition: None - -(32) Project [codegen id : 20] -Output [11]: [ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38] -Input [13]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_date_sk#37, d_year#38] - -(33) Scan parquet default.store -Output [3]: [s_store_sk#40, s_store_name#41, s_zip#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#40, s_store_name#41, s_zip#42] - -(35) Filter [codegen id : 6] -Input [3]: [s_store_sk#40, s_store_name#41, s_zip#42] -Condition : ((isnotnull(s_store_sk#40) AND isnotnull(s_store_name#41)) AND isnotnull(s_zip#42)) - -(36) BroadcastExchange -Input [3]: [s_store_sk#40, s_store_name#41, s_zip#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] - -(37) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#40] -Join condition: None - -(38) Project [codegen id : 20] -Output [12]: [ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42] -Input [14]: [ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_sk#40, s_store_name#41, s_zip#42] - -(39) Scan parquet default.customer -Output [6]: [c_customer_sk#44, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(40) ColumnarToRow [codegen id : 7] -Input [6]: [c_customer_sk#44, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49] - -(41) Filter [codegen id : 7] -Input [6]: [c_customer_sk#44, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49] -Condition : (((((isnotnull(c_customer_sk#44) AND isnotnull(c_first_sales_date_sk#49)) AND isnotnull(c_first_shipto_date_sk#48)) AND isnotnull(c_current_cdemo_sk#45)) AND isnotnull(c_current_hdemo_sk#46)) AND isnotnull(c_current_addr_sk#47)) - -(42) BroadcastExchange -Input [6]: [c_customer_sk#44, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] - -(43) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ss_customer_sk#3] -Right keys [1]: [c_customer_sk#44] -Join condition: None - -(44) Project [codegen id : 20] -Output [16]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49] -Input [18]: [ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_customer_sk#44, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49] - -(45) Scan parquet default.date_dim -Output [2]: [d_date_sk#51, d_year#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#51, d_year#52] - -(47) Filter [codegen id : 8] -Input [2]: [d_date_sk#51, d_year#52] -Condition : isnotnull(d_date_sk#51) - -(48) BroadcastExchange -Input [2]: [d_date_sk#51, d_year#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] - -(49) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [c_first_sales_date_sk#49] -Right keys [1]: [d_date_sk#51] -Join condition: None - -(50) Project [codegen id : 20] -Output [16]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, d_year#52] -Input [18]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49, d_date_sk#51, d_year#52] - -(51) ReusedExchange [Reuses operator id: 48] -Output [2]: [d_date_sk#54, d_year#55] - -(52) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [c_first_shipto_date_sk#48] -Right keys [1]: [d_date_sk#54] -Join condition: None - -(53) Project [codegen id : 20] -Output [16]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#52, d_year#55] -Input [18]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, d_year#52, d_date_sk#54, d_year#55] - -(54) Scan parquet default.customer_demographics -Output [2]: [cd_demo_sk#56, cd_marital_status#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] -ReadSchema: struct - -(55) ColumnarToRow [codegen id : 10] -Input [2]: [cd_demo_sk#56, cd_marital_status#57] - -(56) Filter [codegen id : 10] -Input [2]: [cd_demo_sk#56, cd_marital_status#57] -Condition : (isnotnull(cd_demo_sk#56) AND isnotnull(cd_marital_status#57)) - -(57) BroadcastExchange -Input [2]: [cd_demo_sk#56, cd_marital_status#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] - -(58) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ss_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#56] -Join condition: None - -(59) Project [codegen id : 20] -Output [16]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#52, d_year#55, cd_marital_status#57] -Input [18]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#52, d_year#55, cd_demo_sk#56, cd_marital_status#57] - -(60) ReusedExchange [Reuses operator id: 57] -Output [2]: [cd_demo_sk#59, cd_marital_status#60] - -(61) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [c_current_cdemo_sk#45] -Right keys [1]: [cd_demo_sk#59] -Join condition: NOT (cd_marital_status#57 = cd_marital_status#60) - -(62) Project [codegen id : 20] -Output [14]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#52, d_year#55] -Input [18]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#52, d_year#55, cd_marital_status#57, cd_demo_sk#59, cd_marital_status#60] - -(63) Scan parquet default.promotion -Output [1]: [p_promo_sk#61] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [1]: [p_promo_sk#61] - -(65) Filter [codegen id : 12] -Input [1]: [p_promo_sk#61] -Condition : isnotnull(p_promo_sk#61) - -(66) BroadcastExchange -Input [1]: [p_promo_sk#61] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#62] - -(67) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ss_promo_sk#8] -Right keys [1]: [p_promo_sk#61] -Join condition: None - -(68) Project [codegen id : 20] -Output [13]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#52, d_year#55] -Input [15]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#52, d_year#55, p_promo_sk#61] - -(69) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#63, hd_income_band_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 13] -Input [2]: [hd_demo_sk#63, hd_income_band_sk#64] - -(71) Filter [codegen id : 13] -Input [2]: [hd_demo_sk#63, hd_income_band_sk#64] -Condition : (isnotnull(hd_demo_sk#63) AND isnotnull(hd_income_band_sk#64)) - -(72) BroadcastExchange -Input [2]: [hd_demo_sk#63, hd_income_band_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] - -(73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ss_hdemo_sk#5] -Right keys [1]: [hd_demo_sk#63] -Join condition: None - -(74) Project [codegen id : 20] -Output [13]: [ss_item_sk#2, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#52, d_year#55, hd_income_band_sk#64] -Input [15]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#52, d_year#55, hd_demo_sk#63, hd_income_band_sk#64] - -(75) ReusedExchange [Reuses operator id: 72] -Output [2]: [hd_demo_sk#66, hd_income_band_sk#67] - -(76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [c_current_hdemo_sk#46] -Right keys [1]: [hd_demo_sk#66] -Join condition: None - -(77) Project [codegen id : 20] -Output [13]: [ss_item_sk#2, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_addr_sk#47, d_year#52, d_year#55, hd_income_band_sk#64, hd_income_band_sk#67] -Input [15]: [ss_item_sk#2, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#52, d_year#55, hd_income_band_sk#64, hd_demo_sk#66, hd_income_band_sk#67] - -(78) Scan parquet default.customer_address -Output [5]: [ca_address_sk#68, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(79) ColumnarToRow [codegen id : 15] -Input [5]: [ca_address_sk#68, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72] - -(80) Filter [codegen id : 15] -Input [5]: [ca_address_sk#68, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72] -Condition : isnotnull(ca_address_sk#68) - -(81) BroadcastExchange -Input [5]: [ca_address_sk#68, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] - -(82) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ss_addr_sk#6] -Right keys [1]: [ca_address_sk#68] -Join condition: None - -(83) Project [codegen id : 20] -Output [16]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_addr_sk#47, d_year#52, d_year#55, hd_income_band_sk#64, hd_income_band_sk#67, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72] -Input [18]: [ss_item_sk#2, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_addr_sk#47, d_year#52, d_year#55, hd_income_band_sk#64, hd_income_band_sk#67, ca_address_sk#68, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72] - -(84) ReusedExchange [Reuses operator id: 81] -Output [5]: [ca_address_sk#74, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78] - -(85) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [c_current_addr_sk#47] -Right keys [1]: [ca_address_sk#74] -Join condition: None - -(86) Project [codegen id : 20] -Output [19]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#52, d_year#55, hd_income_band_sk#64, hd_income_band_sk#67, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78] -Input [21]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_addr_sk#47, d_year#52, d_year#55, hd_income_band_sk#64, hd_income_band_sk#67, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_address_sk#74, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78] - -(87) Scan parquet default.income_band -Output [1]: [ib_income_band_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(88) ColumnarToRow [codegen id : 17] -Input [1]: [ib_income_band_sk#79] - -(89) Filter [codegen id : 17] -Input [1]: [ib_income_band_sk#79] -Condition : isnotnull(ib_income_band_sk#79) - -(90) BroadcastExchange -Input [1]: [ib_income_band_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] - -(91) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [hd_income_band_sk#64] -Right keys [1]: [ib_income_band_sk#79] -Join condition: None - -(92) Project [codegen id : 20] -Output [18]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#52, d_year#55, hd_income_band_sk#67, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78] -Input [20]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#52, d_year#55, hd_income_band_sk#64, hd_income_band_sk#67, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, ib_income_band_sk#79] - -(93) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#81] - -(94) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [hd_income_band_sk#67] -Right keys [1]: [ib_income_band_sk#81] -Join condition: None - -(95) Project [codegen id : 20] -Output [17]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#52, d_year#55, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78] -Input [19]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#52, d_year#55, hd_income_band_sk#67, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, ib_income_band_sk#81] - -(96) Scan parquet default.item -Output [4]: [i_item_sk#82, i_current_price#83, i_color#84, i_product_name#85] -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), IsNotNull(i_item_sk)] -ReadSchema: struct - -(97) ColumnarToRow [codegen id : 19] -Input [4]: [i_item_sk#82, i_current_price#83, i_color#84, i_product_name#85] - -(98) Filter [codegen id : 19] -Input [4]: [i_item_sk#82, i_current_price#83, i_color#84, i_product_name#85] -Condition : ((((((isnotnull(i_current_price#83) AND i_color#84 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#83 >= 64.00)) AND (cast(i_current_price#83 as decimal(12,2)) <= 74.00)) AND (cast(i_current_price#83 as decimal(12,2)) >= 65.00)) AND (cast(i_current_price#83 as decimal(12,2)) <= 79.00)) AND isnotnull(i_item_sk#82)) - -(99) Project [codegen id : 19] -Output [2]: [i_item_sk#82, i_product_name#85] -Input [4]: [i_item_sk#82, i_current_price#83, i_color#84, i_product_name#85] - -(100) BroadcastExchange -Input [2]: [i_item_sk#82, i_product_name#85] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#86] - -(101) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#82] -Join condition: None - -(102) Project [codegen id : 20] -Output [18]: [ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, d_year#52, d_year#55, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, i_item_sk#82, i_product_name#85] -Input [19]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#52, d_year#55, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, i_item_sk#82, i_product_name#85] - -(103) HashAggregate [codegen id : 20] -Input [18]: [ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, d_year#52, d_year#55, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, i_item_sk#82, i_product_name#85] -Keys [15]: [i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, d_year#38, d_year#52, d_year#55] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#10)), partial_sum(UnscaledValue(ss_list_price#11)), partial_sum(UnscaledValue(ss_coupon_amt#12))] -Aggregate Attributes [4]: [count#87, sum#88, sum#89, sum#90] -Results [19]: [i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, d_year#38, d_year#52, d_year#55, count#91, sum#92, sum#93, sum#94] - -(104) Exchange -Input [19]: [i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, d_year#38, d_year#52, d_year#55, count#91, sum#92, sum#93, sum#94] -Arguments: hashpartitioning(i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, d_year#38, d_year#52, d_year#55, 5), true, [id=#95] - -(105) HashAggregate [codegen id : 42] -Input [19]: [i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, d_year#38, d_year#52, d_year#55, count#91, sum#92, sum#93, sum#94] -Keys [15]: [i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#75, ca_street_name#76, ca_city#77, ca_zip#78, d_year#38, d_year#52, d_year#55] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#10)), sum(UnscaledValue(ss_list_price#11)), sum(UnscaledValue(ss_coupon_amt#12))] -Aggregate Attributes [4]: [count(1)#96, sum(UnscaledValue(ss_wholesale_cost#10))#97, sum(UnscaledValue(ss_list_price#11))#98, sum(UnscaledValue(ss_coupon_amt#12))#99] -Results [17]: [i_product_name#85 AS product_name#100, i_item_sk#82 AS item_sk#101, s_store_name#41 AS store_name#102, s_zip#42 AS store_zip#103, ca_street_number#69 AS b_street_number#104, ca_street_name#70 AS b_streen_name#105, ca_city#71 AS b_city#106, ca_zip#72 AS b_zip#107, ca_street_number#75 AS c_street_number#108, ca_street_name#76 AS c_street_name#109, ca_city#77 AS c_city#110, ca_zip#78 AS c_zip#111, d_year#38 AS syear#112, count(1)#96 AS cnt#113, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#10))#97,17,2) AS s1#114, MakeDecimal(sum(UnscaledValue(ss_list_price#11))#98,17,2) AS s2#115, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#12))#99,17,2) AS s3#116] - -(106) Scan parquet default.store_sales -Output [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_ticket_number#9, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(107) ColumnarToRow [codegen id : 40] -Input [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_ticket_number#9, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12] - -(108) Filter [codegen id : 40] -Input [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_ticket_number#9, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12] -Condition : ((((((((isnotnull(ss_item_sk#2) AND isnotnull(ss_ticket_number#9)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#7)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_cdemo_sk#4)) AND isnotnull(ss_promo_sk#8)) AND isnotnull(ss_hdemo_sk#5)) AND isnotnull(ss_addr_sk#6)) - -(109) ReusedExchange [Reuses operator id: 7] -Output [2]: [sr_item_sk#13, sr_ticket_number#14] - -(110) BroadcastHashJoin [codegen id : 40] -Left keys [2]: [cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#9 as bigint)] -Right keys [2]: [sr_item_sk#13, sr_ticket_number#14] -Join condition: None - -(111) Project [codegen id : 40] -Output [11]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12] -Input [14]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_ticket_number#9, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, sr_item_sk#13, sr_ticket_number#14] - -(112) ReusedExchange [Reuses operator id: 24] -Output [1]: [cs_item_sk#16] - -(113) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [cs_item_sk#16] -Join condition: None - -(114) Project [codegen id : 40] -Output [11]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12] -Input [12]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, cs_item_sk#16] - -(115) Scan parquet default.date_dim -Output [2]: [d_date_sk#37, d_year#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(116) ColumnarToRow [codegen id : 25] -Input [2]: [d_date_sk#37, d_year#38] - -(117) Filter [codegen id : 25] -Input [2]: [d_date_sk#37, d_year#38] -Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2000)) AND isnotnull(d_date_sk#37)) - -(118) BroadcastExchange -Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#117] - -(119) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#37] -Join condition: None - -(120) Project [codegen id : 40] -Output [11]: [ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38] -Input [13]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_date_sk#37, d_year#38] - -(121) ReusedExchange [Reuses operator id: 36] -Output [3]: [s_store_sk#40, s_store_name#41, s_zip#42] - -(122) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#40] -Join condition: None - -(123) Project [codegen id : 40] -Output [12]: [ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42] -Input [14]: [ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_store_sk#7, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_sk#40, s_store_name#41, s_zip#42] - -(124) ReusedExchange [Reuses operator id: 42] -Output [6]: [c_customer_sk#44, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49] - -(125) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [ss_customer_sk#3] -Right keys [1]: [c_customer_sk#44] -Join condition: None - -(126) Project [codegen id : 40] -Output [16]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49] -Input [18]: [ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_customer_sk#44, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49] - -(127) ReusedExchange [Reuses operator id: 48] -Output [2]: [d_date_sk#118, d_year#119] - -(128) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [c_first_sales_date_sk#49] -Right keys [1]: [d_date_sk#118] -Join condition: None - -(129) Project [codegen id : 40] -Output [16]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, d_year#119] -Input [18]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, c_first_sales_date_sk#49, d_date_sk#118, d_year#119] - -(130) ReusedExchange [Reuses operator id: 48] -Output [2]: [d_date_sk#120, d_year#121] - -(131) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [c_first_shipto_date_sk#48] -Right keys [1]: [d_date_sk#120] -Join condition: None - -(132) Project [codegen id : 40] -Output [16]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#119, d_year#121] -Input [18]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, c_first_shipto_date_sk#48, d_year#119, d_date_sk#120, d_year#121] - -(133) ReusedExchange [Reuses operator id: 57] -Output [2]: [cd_demo_sk#56, cd_marital_status#57] - -(134) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [ss_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#56] -Join condition: None - -(135) Project [codegen id : 40] -Output [16]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#119, d_year#121, cd_marital_status#57] -Input [18]: [ss_item_sk#2, ss_cdemo_sk#4, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#119, d_year#121, cd_demo_sk#56, cd_marital_status#57] - -(136) ReusedExchange [Reuses operator id: 57] -Output [2]: [cd_demo_sk#122, cd_marital_status#123] - -(137) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [c_current_cdemo_sk#45] -Right keys [1]: [cd_demo_sk#122] -Join condition: NOT (cd_marital_status#57 = cd_marital_status#123) - -(138) Project [codegen id : 40] -Output [14]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#119, d_year#121] -Input [18]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_cdemo_sk#45, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#119, d_year#121, cd_marital_status#57, cd_demo_sk#122, cd_marital_status#123] - -(139) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#61] - -(140) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [ss_promo_sk#8] -Right keys [1]: [p_promo_sk#61] -Join condition: None - -(141) Project [codegen id : 40] -Output [13]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#119, d_year#121] -Input [15]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_promo_sk#8, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#119, d_year#121, p_promo_sk#61] - -(142) ReusedExchange [Reuses operator id: 72] -Output [2]: [hd_demo_sk#63, hd_income_band_sk#64] - -(143) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [ss_hdemo_sk#5] -Right keys [1]: [hd_demo_sk#63] -Join condition: None - -(144) Project [codegen id : 40] -Output [13]: [ss_item_sk#2, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#119, d_year#121, hd_income_band_sk#64] -Input [15]: [ss_item_sk#2, ss_hdemo_sk#5, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#119, d_year#121, hd_demo_sk#63, hd_income_band_sk#64] - -(145) ReusedExchange [Reuses operator id: 72] -Output [2]: [hd_demo_sk#124, hd_income_band_sk#125] - -(146) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [c_current_hdemo_sk#46] -Right keys [1]: [hd_demo_sk#124] -Join condition: None - -(147) Project [codegen id : 40] -Output [13]: [ss_item_sk#2, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_addr_sk#47, d_year#119, d_year#121, hd_income_band_sk#64, hd_income_band_sk#125] -Input [15]: [ss_item_sk#2, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_hdemo_sk#46, c_current_addr_sk#47, d_year#119, d_year#121, hd_income_band_sk#64, hd_demo_sk#124, hd_income_band_sk#125] - -(148) ReusedExchange [Reuses operator id: 81] -Output [5]: [ca_address_sk#68, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72] - -(149) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [ss_addr_sk#6] -Right keys [1]: [ca_address_sk#68] -Join condition: None - -(150) Project [codegen id : 40] -Output [16]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_addr_sk#47, d_year#119, d_year#121, hd_income_band_sk#64, hd_income_band_sk#125, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72] -Input [18]: [ss_item_sk#2, ss_addr_sk#6, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_addr_sk#47, d_year#119, d_year#121, hd_income_band_sk#64, hd_income_band_sk#125, ca_address_sk#68, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72] - -(151) ReusedExchange [Reuses operator id: 81] -Output [5]: [ca_address_sk#126, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130] - -(152) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [c_current_addr_sk#47] -Right keys [1]: [ca_address_sk#126] -Join condition: None - -(153) Project [codegen id : 40] -Output [19]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#119, d_year#121, hd_income_band_sk#64, hd_income_band_sk#125, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130] -Input [21]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, c_current_addr_sk#47, d_year#119, d_year#121, hd_income_band_sk#64, hd_income_band_sk#125, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_address_sk#126, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130] - -(154) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#79] - -(155) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [hd_income_band_sk#64] -Right keys [1]: [ib_income_band_sk#79] -Join condition: None - -(156) Project [codegen id : 40] -Output [18]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#119, d_year#121, hd_income_band_sk#125, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130] -Input [20]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#119, d_year#121, hd_income_band_sk#64, hd_income_band_sk#125, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, ib_income_band_sk#79] - -(157) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#131] - -(158) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [hd_income_band_sk#125] -Right keys [1]: [ib_income_band_sk#131] -Join condition: None - -(159) Project [codegen id : 40] -Output [17]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#119, d_year#121, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130] -Input [19]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#119, d_year#121, hd_income_band_sk#125, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, ib_income_band_sk#131] - -(160) ReusedExchange [Reuses operator id: 100] -Output [2]: [i_item_sk#82, i_product_name#85] - -(161) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#82] -Join condition: None - -(162) Project [codegen id : 40] -Output [18]: [ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, d_year#119, d_year#121, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, i_item_sk#82, i_product_name#85] -Input [19]: [ss_item_sk#2, ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, s_store_name#41, s_zip#42, d_year#119, d_year#121, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, i_item_sk#82, i_product_name#85] - -(163) HashAggregate [codegen id : 40] -Input [18]: [ss_wholesale_cost#10, ss_list_price#11, ss_coupon_amt#12, d_year#38, d_year#119, d_year#121, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, i_item_sk#82, i_product_name#85] -Keys [15]: [i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, d_year#38, d_year#119, d_year#121] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#10)), partial_sum(UnscaledValue(ss_list_price#11)), partial_sum(UnscaledValue(ss_coupon_amt#12))] -Aggregate Attributes [4]: [count#132, sum#133, sum#134, sum#135] -Results [19]: [i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, d_year#38, d_year#119, d_year#121, count#136, sum#137, sum#138, sum#139] - -(164) Exchange -Input [19]: [i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, d_year#38, d_year#119, d_year#121, count#136, sum#137, sum#138, sum#139] -Arguments: hashpartitioning(i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, d_year#38, d_year#119, d_year#121, 5), true, [id=#140] - -(165) HashAggregate [codegen id : 41] -Input [19]: [i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, d_year#38, d_year#119, d_year#121, count#136, sum#137, sum#138, sum#139] -Keys [15]: [i_product_name#85, i_item_sk#82, s_store_name#41, s_zip#42, ca_street_number#69, ca_street_name#70, ca_city#71, ca_zip#72, ca_street_number#127, ca_street_name#128, ca_city#129, ca_zip#130, d_year#38, d_year#119, d_year#121] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#10)), sum(UnscaledValue(ss_list_price#11)), sum(UnscaledValue(ss_coupon_amt#12))] -Aggregate Attributes [4]: [count(1)#141, sum(UnscaledValue(ss_wholesale_cost#10))#142, sum(UnscaledValue(ss_list_price#11))#143, sum(UnscaledValue(ss_coupon_amt#12))#144] -Results [8]: [i_item_sk#82 AS item_sk#145, s_store_name#41 AS store_name#146, s_zip#42 AS store_zip#147, d_year#38 AS syear#148, count(1)#141 AS cnt#149, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#10))#142,17,2) AS s1#150, MakeDecimal(sum(UnscaledValue(ss_list_price#11))#143,17,2) AS s2#151, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#12))#144,17,2) AS s3#152] - -(166) BroadcastExchange -Input [8]: [item_sk#145, store_name#146, store_zip#147, syear#148, cnt#149, s1#150, s2#151, s3#152] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, string, true], input[2, string, true]),false), [id=#153] - -(167) BroadcastHashJoin [codegen id : 42] -Left keys [3]: [item_sk#101, store_name#102, store_zip#103] -Right keys [3]: [item_sk#145, store_name#146, store_zip#147] -Join condition: (cnt#149 <= cnt#113) - -(168) Project [codegen id : 42] -Output [21]: [product_name#100, store_name#102, store_zip#103, b_street_number#104, b_streen_name#105, b_city#106, b_zip#107, c_street_number#108, c_street_name#109, c_city#110, c_zip#111, syear#112, cnt#113, s1#114, s2#115, s3#116, s1#150, s2#151, s3#152, syear#148, cnt#149] -Input [25]: [product_name#100, item_sk#101, store_name#102, store_zip#103, b_street_number#104, b_streen_name#105, b_city#106, b_zip#107, c_street_number#108, c_street_name#109, c_city#110, c_zip#111, syear#112, cnt#113, s1#114, s2#115, s3#116, item_sk#145, store_name#146, store_zip#147, syear#148, cnt#149, s1#150, s2#151, s3#152] - -(169) Exchange -Input [21]: [product_name#100, store_name#102, store_zip#103, b_street_number#104, b_streen_name#105, b_city#106, b_zip#107, c_street_number#108, c_street_name#109, c_city#110, c_zip#111, syear#112, cnt#113, s1#114, s2#115, s3#116, s1#150, s2#151, s3#152, syear#148, cnt#149] -Arguments: rangepartitioning(product_name#100 ASC NULLS FIRST, store_name#102 ASC NULLS FIRST, cnt#149 ASC NULLS FIRST, 5), true, [id=#154] - -(170) Sort [codegen id : 43] -Input [21]: [product_name#100, store_name#102, store_zip#103, b_street_number#104, b_streen_name#105, b_city#106, b_zip#107, c_street_number#108, c_street_name#109, c_city#110, c_zip#111, syear#112, cnt#113, s1#114, s2#115, s3#116, s1#150, s2#151, s3#152, syear#148, cnt#149] -Arguments: [product_name#100 ASC NULLS FIRST, store_name#102 ASC NULLS FIRST, cnt#149 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q64/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q64/simplified.txt deleted file mode 100644 index 4c40a359c..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q64/simplified.txt +++ /dev/null @@ -1,246 +0,0 @@ -WholeStageCodegen (43) - Sort [product_name,store_name,cnt] - InputAdapter - Exchange [product_name,store_name,cnt] #1 - WholeStageCodegen (42) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - BroadcastHashJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - InputAdapter - Exchange [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year] #2 - WholeStageCodegen (20) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - BroadcastHashJoin [ss_item_sk,cs_item_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - Filter [ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [cs_item_sk] - Filter [sum(cs_ext_list_price),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true))] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(cs_ext_list_price),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - BroadcastHashJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - Filter [cs_item_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [cr_item_sk,cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - Filter [s_store_sk,s_store_name,s_zip] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - Filter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #10 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (10) - Filter [cd_demo_sk,cd_marital_status] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (12) - Filter [p_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.promotion [p_promo_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (13) - Filter [hd_demo_sk,hd_income_band_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_income_band_sk] - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #13 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (15) - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (17) - Filter [ib_income_band_sk] - ColumnarToRow - InputAdapter - Scan parquet default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #15 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (19) - Project [i_item_sk,i_product_name] - Filter [i_current_price,i_color,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (41) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - InputAdapter - Exchange [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year] #18 - WholeStageCodegen (40) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - BroadcastHashJoin [ss_item_sk,cs_item_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - Filter [ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt] - InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number] #3 - InputAdapter - ReusedExchange [cs_item_sk] #4 - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (25) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #8 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #9 - InputAdapter - ReusedExchange [d_date_sk,d_year] #10 - InputAdapter - ReusedExchange [d_date_sk,d_year] #10 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #11 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #11 - InputAdapter - ReusedExchange [p_promo_sk] #12 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #13 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #13 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 - InputAdapter - ReusedExchange [ib_income_band_sk] #15 - InputAdapter - ReusedExchange [ib_income_band_sk] #15 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #16 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q65/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q65/explain.txt deleted file mode 100644 index ab87816b8..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q65/explain.txt +++ /dev/null @@ -1,245 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.store (1) - : : +- BroadcastExchange (18) - : : +- * Filter (17) - : : +- * HashAggregate (16) - : : +- Exchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.store_sales (4) - : : +- BroadcastExchange (11) - : : +- * Project (10) - : : +- * Filter (9) - : : +- * ColumnarToRow (8) - : : +- Scan parquet default.date_dim (7) - : +- BroadcastExchange (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet default.item (21) - +- BroadcastExchange (39) - +- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.store_sales (27) - +- ReusedExchange (30) - - -(1) Scan parquet default.store -Output [2]: [s_store_sk#1, s_store_name#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#1, s_store_name#2] - -(3) Filter [codegen id : 9] -Input [2]: [s_store_sk#1, s_store_name#2] -Condition : isnotnull(s_store_sk#1) - -(4) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [4]: [ss_sold_date_sk#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6] - -(6) Filter [codegen id : 2] -Input [4]: [ss_sold_date_sk#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6] -Condition : ((isnotnull(ss_sold_date_sk#3) AND isnotnull(ss_store_sk#5)) AND isnotnull(ss_item_sk#4)) - -(7) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_month_seq#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#7, d_month_seq#8] - -(9) Filter [codegen id : 1] -Input [2]: [d_date_sk#7, d_month_seq#8] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1176)) AND (d_month_seq#8 <= 1187)) AND isnotnull(d_date_sk#7)) - -(10) Project [codegen id : 1] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_month_seq#8] - -(11) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] - -(12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#7] -Join condition: None - -(13) Project [codegen id : 2] -Output [3]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6] -Input [5]: [ss_sold_date_sk#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, d_date_sk#7] - -(14) HashAggregate [codegen id : 2] -Input [3]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6] -Keys [2]: [ss_store_sk#5, ss_item_sk#4] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [1]: [sum#10] -Results [3]: [ss_store_sk#5, ss_item_sk#4, sum#11] - -(15) Exchange -Input [3]: [ss_store_sk#5, ss_item_sk#4, sum#11] -Arguments: hashpartitioning(ss_store_sk#5, ss_item_sk#4, 5), true, [id=#12] - -(16) HashAggregate [codegen id : 3] -Input [3]: [ss_store_sk#5, ss_item_sk#4, sum#11] -Keys [2]: [ss_store_sk#5, ss_item_sk#4] -Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#13] -Results [3]: [ss_store_sk#5, ss_item_sk#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#13,17,2) AS revenue#14] - -(17) Filter [codegen id : 3] -Input [3]: [ss_store_sk#5, ss_item_sk#4, revenue#14] -Condition : isnotnull(revenue#14) - -(18) BroadcastExchange -Input [3]: [ss_store_sk#5, ss_item_sk#4, revenue#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] - -(19) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [s_store_sk#1] -Right keys [1]: [ss_store_sk#5] -Join condition: None - -(20) Project [codegen id : 9] -Output [4]: [s_store_name#2, ss_store_sk#5, ss_item_sk#4, revenue#14] -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#5, ss_item_sk#4, revenue#14] - -(21) Scan parquet default.item -Output [5]: [i_item_sk#16, i_item_desc#17, i_current_price#18, i_wholesale_cost#19, i_brand#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) ColumnarToRow [codegen id : 4] -Input [5]: [i_item_sk#16, i_item_desc#17, i_current_price#18, i_wholesale_cost#19, i_brand#20] - -(23) Filter [codegen id : 4] -Input [5]: [i_item_sk#16, i_item_desc#17, i_current_price#18, i_wholesale_cost#19, i_brand#20] -Condition : isnotnull(i_item_sk#16) - -(24) BroadcastExchange -Input [5]: [i_item_sk#16, i_item_desc#17, i_current_price#18, i_wholesale_cost#19, i_brand#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] - -(25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join condition: None - -(26) Project [codegen id : 9] -Output [7]: [s_store_name#2, ss_store_sk#5, revenue#14, i_item_desc#17, i_current_price#18, i_wholesale_cost#19, i_brand#20] -Input [9]: [s_store_name#2, ss_store_sk#5, ss_item_sk#4, revenue#14, i_item_sk#16, i_item_desc#17, i_current_price#18, i_wholesale_cost#19, i_brand#20] - -(27) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#22, ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(28) ColumnarToRow [codegen id : 6] -Input [4]: [ss_sold_date_sk#22, ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] - -(29) Filter [codegen id : 6] -Input [4]: [ss_sold_date_sk#22, ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Condition : (isnotnull(ss_sold_date_sk#22) AND isnotnull(ss_store_sk#24)) - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#7] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#7] -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_sold_date_sk#22, ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, d_date_sk#7] - -(33) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Keys [2]: [ss_store_sk#24, ss_item_sk#23] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#26] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#27] - -(34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#27] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), true, [id=#28] - -(35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#27] -Keys [2]: [ss_store_sk#24, ss_item_sk#23] -Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#29] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#29,17,2) AS revenue#30] - -(36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#30] -Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#30)] -Aggregate Attributes [2]: [sum#31, count#32] -Results [3]: [ss_store_sk#24, sum#33, count#34] - -(37) Exchange -Input [3]: [ss_store_sk#24, sum#33, count#34] -Arguments: hashpartitioning(ss_store_sk#24, 5), true, [id=#35] - -(38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#33, count#34] -Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#30)] -Aggregate Attributes [1]: [avg(revenue#30)#36] -Results [2]: [ss_store_sk#24, avg(revenue#30)#36 AS ave#37] - -(39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] - -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#14 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#37)), DecimalType(23,7), true)) - -(41) Project [codegen id : 9] -Output [6]: [s_store_name#2, i_item_desc#17, revenue#14, i_current_price#18, i_wholesale_cost#19, i_brand#20] -Input [9]: [s_store_name#2, ss_store_sk#5, revenue#14, i_item_desc#17, i_current_price#18, i_wholesale_cost#19, i_brand#20, ss_store_sk#24, ave#37] - -(42) TakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#17, revenue#14, i_current_price#18, i_wholesale_cost#19, i_brand#20] -Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#17 ASC NULLS FIRST], [s_store_name#2, i_item_desc#17, revenue#14, i_current_price#18, i_wholesale_cost#19, i_brand#20] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q65/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q65/simplified.txt deleted file mode 100644 index a4b468ffe..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q65/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (9) - Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] - Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [s_store_name,ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #1 - WholeStageCodegen (3) - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - InputAdapter - Exchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - InputAdapter - Exchange [ss_store_sk] #6 - WholeStageCodegen (7) - HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - InputAdapter - Exchange [ss_store_sk,ss_item_sk] #7 - WholeStageCodegen (6) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q66/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q66/explain.txt deleted file mode 100644 index fc18efd3d..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q66/explain.txt +++ /dev/null @@ -1,310 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (55) -+- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- Union (51) - :- * HashAggregate (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.warehouse (4) - : : : +- BroadcastExchange (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (20) - : : +- * Project (19) - : : +- * Filter (18) - : : +- * ColumnarToRow (17) - : : +- Scan parquet default.time_dim (16) - : +- BroadcastExchange (27) - : +- * Project (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.ship_mode (23) - +- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (41) - : : +- * BroadcastHashJoin Inner BuildRight (40) - : : :- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.catalog_sales (33) - : : : +- ReusedExchange (36) - : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- ReusedExchange (45) - - -(1) Scan parquet default.web_sales -Output [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_date_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] - -(3) Filter [codegen id : 5] -Input [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] -Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_sold_date_sk#1)) AND isnotnull(ws_sold_time_sk#2)) AND isnotnull(ws_ship_mode_sk#3)) - -(4) Scan parquet default.warehouse -Output [7]: [w_warehouse_sk#8, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [7]: [w_warehouse_sk#8, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14] - -(6) Filter [codegen id : 1] -Input [7]: [w_warehouse_sk#8, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14] -Condition : isnotnull(w_warehouse_sk#8) - -(7) BroadcastExchange -Input [7]: [w_warehouse_sk#8, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#4] -Right keys [1]: [w_warehouse_sk#8] -Join condition: None - -(9) Project [codegen id : 5] -Output [12]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14] -Input [14]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_sk#8, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14] - -(10) Scan parquet default.date_dim -Output [3]: [d_date_sk#16, d_year#17, d_moy#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#16, d_year#17, d_moy#18] - -(12) Filter [codegen id : 2] -Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) - -(13) BroadcastExchange -Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] - -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#16] -Join condition: None - -(15) Project [codegen id : 5] -Output [13]: [ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18] -Input [15]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_date_sk#16, d_year#17, d_moy#18] - -(16) Scan parquet default.time_dim -Output [2]: [t_time_sk#20, t_time#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [2]: [t_time_sk#20, t_time#21] - -(18) Filter [codegen id : 3] -Input [2]: [t_time_sk#20, t_time#21] -Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= 59638)) AND isnotnull(t_time_sk#20)) - -(19) Project [codegen id : 3] -Output [1]: [t_time_sk#20] -Input [2]: [t_time_sk#20, t_time#21] - -(20) BroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_time_sk#2] -Right keys [1]: [t_time_sk#20] -Join condition: None - -(22) Project [codegen id : 5] -Output [12]: [ws_ship_mode_sk#3, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18] -Input [14]: [ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18, t_time_sk#20] - -(23) Scan parquet default.ship_mode -Output [2]: [sm_ship_mode_sk#23, sm_carrier#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [In(sm_carrier, [DHL,BARIAN]), IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [sm_ship_mode_sk#23, sm_carrier#24] - -(25) Filter [codegen id : 4] -Input [2]: [sm_ship_mode_sk#23, sm_carrier#24] -Condition : (sm_carrier#24 IN (DHL,BARIAN) AND isnotnull(sm_ship_mode_sk#23)) - -(26) Project [codegen id : 4] -Output [1]: [sm_ship_mode_sk#23] -Input [2]: [sm_ship_mode_sk#23, sm_carrier#24] - -(27) BroadcastExchange -Input [1]: [sm_ship_mode_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] - -(28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#3] -Right keys [1]: [sm_ship_mode_sk#23] -Join condition: None - -(29) Project [codegen id : 5] -Output [11]: [ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18] -Input [13]: [ws_ship_mode_sk#3, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18, sm_ship_mode_sk#23] - -(30) HashAggregate [codegen id : 5] -Input [11]: [ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18] -Keys [7]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17] -Functions [24]: [partial_sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Results [55]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121] - -(31) Exchange -Input [55]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121] -Arguments: hashpartitioning(w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, 5), true, [id=#122] - -(32) HashAggregate [codegen id : 6] -Input [55]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121] -Keys [7]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17] -Functions [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#142, sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#143, sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#144, sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#145, sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#146] -Results [32]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, DHL,BARIAN AS ship_carriers#147, d_year#17 AS year#148, sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#123 AS jan_sales#149, sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#124 AS feb_sales#150, sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#125 AS mar_sales#151, sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#126 AS apr_sales#152, sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#127 AS may_sales#153, sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#128 AS jun_sales#154, sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#129 AS jul_sales#155, sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#130 AS aug_sales#156, sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#131 AS sep_sales#157, sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#132 AS oct_sales#158, sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#133 AS nov_sales#159, sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#134 AS dec_sales#160, sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#135 AS jan_net#161, sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#136 AS feb_net#162, sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#137 AS mar_net#163, sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#138 AS apr_net#164, sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#139 AS may_net#165, sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#140 AS jun_net#166, sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#141 AS jul_net#167, sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#142 AS aug_net#168, sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#143 AS sep_net#169, sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#144 AS oct_net#170, sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#145 AS nov_net#171, sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#146 AS dec_net#172] - -(33) Scan parquet default.catalog_sales -Output [7]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 11] -Input [7]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] - -(35) Filter [codegen id : 11] -Input [7]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] -Condition : (((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_date_sk#173)) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) - -(36) ReusedExchange [Reuses operator id: 7] -Output [7]: [w_warehouse_sk#8, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14] - -(37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#8] -Join condition: None - -(38) Project [codegen id : 11] -Output [12]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14] -Input [14]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_sk#8, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14] - -(39) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#16, d_year#17, d_moy#18] - -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#173] -Right keys [1]: [d_date_sk#16] -Join condition: None - -(41) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18] -Input [15]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_date_sk#16, d_year#17, d_moy#18] - -(42) ReusedExchange [Reuses operator id: 20] -Output [1]: [t_time_sk#20] - -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#20] -Join condition: None - -(44) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18, t_time_sk#20] - -(45) ReusedExchange [Reuses operator id: 27] -Output [1]: [sm_ship_mode_sk#23] - -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#23] -Join condition: None - -(47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18] -Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18, sm_ship_mode_sk#23] - -(48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, d_moy#18] -Keys [7]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17] -Functions [24]: [partial_sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227] -Results [55]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263, sum#264, isEmpty#265, sum#266, isEmpty#267, sum#268, isEmpty#269, sum#270, isEmpty#271, sum#272, isEmpty#273, sum#274, isEmpty#275] - -(49) Exchange -Input [55]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263, sum#264, isEmpty#265, sum#266, isEmpty#267, sum#268, isEmpty#269, sum#270, isEmpty#271, sum#272, isEmpty#273, sum#274, isEmpty#275] -Arguments: hashpartitioning(w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, 5), true, [id=#276] - -(50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263, sum#264, isEmpty#265, sum#266, isEmpty#267, sum#268, isEmpty#269, sum#270, isEmpty#271, sum#272, isEmpty#273, sum#274, isEmpty#275] -Keys [7]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, d_year#17] -Functions [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#277, sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300] -Results [32]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, DHL,BARIAN AS ship_carriers#301, d_year#17 AS year#302, sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#277 AS jan_sales#303, sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS feb_sales#304, sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS mar_sales#305, sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS apr_sales#306, sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS may_sales#307, sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS jun_sales#308, sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jul_sales#309, sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS aug_sales#310, sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS sep_sales#311, sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS oct_sales#312, sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS nov_sales#313, sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS dec_sales#314, sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS jan_net#315, sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS feb_net#316, sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS mar_net#317, sum(CASE WHEN (d_moy#18 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS apr_net#318, sum(CASE WHEN (d_moy#18 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS may_net#319, sum(CASE WHEN (d_moy#18 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS jun_net#320, sum(CASE WHEN (d_moy#18 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jul_net#321, sum(CASE WHEN (d_moy#18 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS aug_net#322, sum(CASE WHEN (d_moy#18 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS sep_net#323, sum(CASE WHEN (d_moy#18 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS oct_net#324, sum(CASE WHEN (d_moy#18 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS nov_net#325, sum(CASE WHEN (d_moy#18 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS dec_net#326] - -(51) Union - -(52) HashAggregate [codegen id : 13] -Input [32]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, ship_carriers#147, year#148, jan_sales#149, feb_sales#150, mar_sales#151, apr_sales#152, may_sales#153, jun_sales#154, jul_sales#155, aug_sales#156, sep_sales#157, oct_sales#158, nov_sales#159, dec_sales#160, jan_net#161, feb_net#162, mar_net#163, apr_net#164, may_net#165, jun_net#166, jul_net#167, aug_net#168, sep_net#169, oct_net#170, nov_net#171, dec_net#172] -Keys [8]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, ship_carriers#147, year#148] -Functions [36]: [partial_sum(jan_sales#149), partial_sum(feb_sales#150), partial_sum(mar_sales#151), partial_sum(apr_sales#152), partial_sum(may_sales#153), partial_sum(jun_sales#154), partial_sum(jul_sales#155), partial_sum(aug_sales#156), partial_sum(sep_sales#157), partial_sum(oct_sales#158), partial_sum(nov_sales#159), partial_sum(dec_sales#160), partial_sum(CheckOverflow((promote_precision(jan_sales#149) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#161), partial_sum(feb_net#162), partial_sum(mar_net#163), partial_sum(apr_net#164), partial_sum(may_net#165), partial_sum(jun_net#166), partial_sum(jul_net#167), partial_sum(aug_net#168), partial_sum(sep_net#169), partial_sum(oct_net#170), partial_sum(nov_net#171), partial_sum(dec_net#172)] -Aggregate Attributes [72]: [sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384, sum#385, isEmpty#386, sum#387, isEmpty#388, sum#389, isEmpty#390, sum#391, isEmpty#392, sum#393, isEmpty#394, sum#395, isEmpty#396, sum#397, isEmpty#398] -Results [80]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, ship_carriers#147, year#148, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470] - -(53) Exchange -Input [80]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, ship_carriers#147, year#148, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470] -Arguments: hashpartitioning(w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, ship_carriers#147, year#148, 5), true, [id=#471] - -(54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, ship_carriers#147, year#148, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470] -Keys [8]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, ship_carriers#147, year#148] -Functions [36]: [sum(jan_sales#149), sum(feb_sales#150), sum(mar_sales#151), sum(apr_sales#152), sum(may_sales#153), sum(jun_sales#154), sum(jul_sales#155), sum(aug_sales#156), sum(sep_sales#157), sum(oct_sales#158), sum(nov_sales#159), sum(dec_sales#160), sum(CheckOverflow((promote_precision(jan_sales#149) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#161), sum(feb_net#162), sum(mar_net#163), sum(apr_net#164), sum(may_net#165), sum(jun_net#166), sum(jul_net#167), sum(aug_net#168), sum(sep_net#169), sum(oct_net#170), sum(nov_net#171), sum(dec_net#172)] -Aggregate Attributes [36]: [sum(jan_sales#149)#472, sum(feb_sales#150)#473, sum(mar_sales#151)#474, sum(apr_sales#152)#475, sum(may_sales#153)#476, sum(jun_sales#154)#477, sum(jul_sales#155)#478, sum(aug_sales#156)#479, sum(sep_sales#157)#480, sum(oct_sales#158)#481, sum(nov_sales#159)#482, sum(dec_sales#160)#483, sum(CheckOverflow((promote_precision(jan_sales#149) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#484, sum(CheckOverflow((promote_precision(feb_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(mar_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(apr_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(may_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(jun_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jul_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(aug_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(sep_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(oct_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(nov_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(dec_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(jan_net#161)#496, sum(feb_net#162)#497, sum(mar_net#163)#498, sum(apr_net#164)#499, sum(may_net#165)#500, sum(jun_net#166)#501, sum(jul_net#167)#502, sum(aug_net#168)#503, sum(sep_net#169)#504, sum(oct_net#170)#505, sum(nov_net#171)#506, sum(dec_net#172)#507] -Results [44]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, ship_carriers#147, year#148, sum(jan_sales#149)#472 AS jan_sales#508, sum(feb_sales#150)#473 AS feb_sales#509, sum(mar_sales#151)#474 AS mar_sales#510, sum(apr_sales#152)#475 AS apr_sales#511, sum(may_sales#153)#476 AS may_sales#512, sum(jun_sales#154)#477 AS jun_sales#513, sum(jul_sales#155)#478 AS jul_sales#514, sum(aug_sales#156)#479 AS aug_sales#515, sum(sep_sales#157)#480 AS sep_sales#516, sum(oct_sales#158)#481 AS oct_sales#517, sum(nov_sales#159)#482 AS nov_sales#518, sum(dec_sales#160)#483 AS dec_sales#519, sum(CheckOverflow((promote_precision(jan_sales#149) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#484 AS jan_sales_per_sq_foot#520, sum(CheckOverflow((promote_precision(feb_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS feb_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(mar_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS mar_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(apr_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS apr_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(may_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS may_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(jun_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS jun_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jul_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jul_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(aug_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS aug_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(sep_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS sep_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(oct_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS oct_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(nov_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS nov_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(dec_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#10 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS dec_sales_per_sq_foot#531, sum(jan_net#161)#496 AS jan_net#532, sum(feb_net#162)#497 AS feb_net#533, sum(mar_net#163)#498 AS mar_net#534, sum(apr_net#164)#499 AS apr_net#535, sum(may_net#165)#500 AS may_net#536, sum(jun_net#166)#501 AS jun_net#537, sum(jul_net#167)#502 AS jul_net#538, sum(aug_net#168)#503 AS aug_net#539, sum(sep_net#169)#504 AS sep_net#540, sum(oct_net#170)#505 AS oct_net#541, sum(nov_net#171)#506 AS nov_net#542, sum(dec_net#172)#507 AS dec_net#543] - -(55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, ship_carriers#147, year#148, jan_sales#508, feb_sales#509, mar_sales#510, apr_sales#511, may_sales#512, jun_sales#513, jul_sales#514, aug_sales#515, sep_sales#516, oct_sales#517, nov_sales#518, dec_sales#519, jan_sales_per_sq_foot#520, feb_sales_per_sq_foot#521, mar_sales_per_sq_foot#522, apr_sales_per_sq_foot#523, may_sales_per_sq_foot#524, jun_sales_per_sq_foot#525, jul_sales_per_sq_foot#526, aug_sales_per_sq_foot#527, sep_sales_per_sq_foot#528, oct_sales_per_sq_foot#529, nov_sales_per_sq_foot#530, dec_sales_per_sq_foot#531, jan_net#532, feb_net#533, mar_net#534, apr_net#535, may_net#536, jun_net#537, jul_net#538, aug_net#539, sep_net#540, oct_net#541, nov_net#542, dec_net#543] -Arguments: 100, [w_warehouse_name#9 ASC NULLS FIRST], [w_warehouse_name#9, w_warehouse_sq_ft#10, w_city#11, w_county#12, w_state#13, w_country#14, ship_carriers#147, year#148, jan_sales#508, feb_sales#509, mar_sales#510, apr_sales#511, may_sales#512, jun_sales#513, jul_sales#514, aug_sales#515, sep_sales#516, oct_sales#517, nov_sales#518, dec_sales#519, jan_sales_per_sq_foot#520, feb_sales_per_sq_foot#521, mar_sales_per_sq_foot#522, apr_sales_per_sq_foot#523, ... 20 more fields] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q66/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q66/simplified.txt deleted file mode 100644 index ac7379973..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q66/simplified.txt +++ /dev/null @@ -1,83 +0,0 @@ -TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum(CheckOverflow((promote_precision(jan_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(feb_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(mar_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(apr_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(may_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jun_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jul_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(aug_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(sep_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(oct_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(nov_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(dec_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - Filter [ws_warehouse_sk,ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [t_time_sk] - Filter [t_time,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_time] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Project [sm_ship_mode_sk] - Filter [sm_carrier,sm_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Filter [cs_warehouse_sk,cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #3 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - ReusedExchange [t_time_sk] #5 - InputAdapter - ReusedExchange [sm_ship_mode_sk] #6 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q67/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q67/explain.txt deleted file mode 100644 index ae133938b..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q67/explain.txt +++ /dev/null @@ -1,175 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (31) -+- * Filter (30) - +- Window (29) - +- * Sort (28) - +- Exchange (27) - +- * HashAggregate (26) - +- Exchange (25) - +- * HashAggregate (24) - +- * Expand (23) - +- * Project (22) - +- * BroadcastHashJoin Inner BuildRight (21) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.date_dim (4) - : +- BroadcastExchange (14) - : +- * Filter (13) - : +- * ColumnarToRow (12) - : +- Scan parquet default.store (11) - +- BroadcastExchange (20) - +- * Filter (19) - +- * ColumnarToRow (18) - +- Scan parquet default.item (17) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5] -Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#2)) - -(4) Scan parquet default.date_dim -Output [5]: [d_date_sk#6, d_month_seq#7, d_year#8, d_moy#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [d_date_sk#6, d_month_seq#7, d_year#8, d_moy#9, d_qoy#10] - -(6) Filter [codegen id : 1] -Input [5]: [d_date_sk#6, d_month_seq#7, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(7) Project [codegen id : 1] -Output [4]: [d_date_sk#6, d_year#8, d_moy#9, d_qoy#10] -Input [5]: [d_date_sk#6, d_month_seq#7, d_year#8, d_moy#9, d_qoy#10] - -(8) BroadcastExchange -Input [4]: [d_date_sk#6, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#6] -Join condition: None - -(10) Project [codegen id : 4] -Output [7]: [ss_item_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, d_date_sk#6, d_year#8, d_moy#9, d_qoy#10] - -(11) Scan parquet default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#12, s_store_id#13] - -(13) Filter [codegen id : 2] -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(14) BroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] -Join condition: None - -(16) Project [codegen id : 4] -Output [7]: [ss_item_sk#2, ss_quantity#4, ss_sales_price#5, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] -Input [9]: [ss_item_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, d_year#8, d_moy#9, d_qoy#10, s_store_sk#12, s_store_id#13] - -(17) Scan parquet default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] - -(19) Filter [codegen id : 3] -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(20) BroadcastExchange -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#15] -Join condition: None - -(22) Project [codegen id : 4] -Output [10]: [ss_quantity#4, ss_sales_price#5, i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Input [12]: [ss_item_sk#2, ss_quantity#4, ss_sales_price#5, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] - -(23) Expand [codegen id : 4] -Input [10]: [ss_quantity#4, ss_sales_price#5, i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Arguments: [List(ss_quantity#4, ss_sales_price#5, i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 0), List(ss_quantity#4, ss_sales_price#5, i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, null, 1), List(ss_quantity#4, ss_sales_price#5, i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, null, null, 3), List(ss_quantity#4, ss_sales_price#5, i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, null, null, null, 7), List(ss_quantity#4, ss_sales_price#5, i_category#18, i_class#17, i_brand#16, i_product_name#19, null, null, null, null, 15), List(ss_quantity#4, ss_sales_price#5, i_category#18, i_class#17, i_brand#16, null, null, null, null, null, 31), List(ss_quantity#4, ss_sales_price#5, i_category#18, i_class#17, null, null, null, null, null, null, 63), List(ss_quantity#4, ss_sales_price#5, i_category#18, null, null, null, null, null, null, null, 127), List(ss_quantity#4, ss_sales_price#5, null, null, null, null, null, null, null, null, 255)], [ss_quantity#4, ss_sales_price#5, i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, spark_grouping_id#29] - -(24) HashAggregate [codegen id : 4] -Input [11]: [ss_quantity#4, ss_sales_price#5, i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, spark_grouping_id#29] -Keys [9]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, spark_grouping_id#29] -Functions [1]: [partial_sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#5 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#4 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [2]: [sum#30, isEmpty#31] -Results [11]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, spark_grouping_id#29, sum#32, isEmpty#33] - -(25) Exchange -Input [11]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, spark_grouping_id#29, sum#32, isEmpty#33] -Arguments: hashpartitioning(i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, spark_grouping_id#29, 5), true, [id=#34] - -(26) HashAggregate [codegen id : 5] -Input [11]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, spark_grouping_id#29, sum#32, isEmpty#33] -Keys [9]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, spark_grouping_id#29] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#5 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#4 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#5 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#4 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#35] -Results [9]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#5 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#4 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#35 AS sumsales#36] - -(27) Exchange -Input [9]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36] -Arguments: hashpartitioning(i_category#21, 5), true, [id=#37] - -(28) Sort [codegen id : 6] -Input [9]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36] -Arguments: [i_category#21 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST], false, 0 - -(29) Window -Input [9]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36] -Arguments: [rank(sumsales#36) windowspecdefinition(i_category#21, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [i_category#21], [sumsales#36 DESC NULLS LAST] - -(30) Filter [codegen id : 7] -Input [10]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36, rk#38] -Condition : (isnotnull(rk#38) AND (rk#38 <= 100)) - -(31) TakeOrderedAndProject -Input [10]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36, rk#38] -Arguments: 100, [i_category#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_qoy#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#38 ASC NULLS FIRST], [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36, rk#38] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q67/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q67/simplified.txt deleted file mode 100644 index b343c2d02..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q67/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (7) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (6) - Sort [i_category,sumsales] - InputAdapter - Exchange [i_category] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] - InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_year,d_moy,d_qoy] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q68/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q68/explain.txt deleted file mode 100644 index d0c618bdb..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q68/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * HashAggregate (33) - : : +- Exchange (32) - : : +- * HashAggregate (31) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet default.store_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * Project (7) - : : : : : +- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.date_dim (4) - : : : : +- BroadcastExchange (15) - : : : : +- * Project (14) - : : : : +- * Filter (13) - : : : : +- * ColumnarToRow (12) - : : : : +- Scan parquet default.store (11) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet default.household_demographics (18) - : : +- BroadcastExchange (28) - : : +- * Filter (27) - : : +- * ColumnarToRow (26) - : : +- Scan parquet default.customer_address (25) - : +- BroadcastExchange (37) - : +- * Filter (36) - : +- * ColumnarToRow (35) - : +- Scan parquet default.customer (34) - +- ReusedExchange (40) - - -(1) Scan parquet default.store_sales -Output [9]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [9]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9] - -(3) Filter [codegen id : 5] -Input [9]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9] -Condition : ((((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#5)) AND isnotnull(ss_hdemo_sk#3)) AND isnotnull(ss_addr_sk#4)) AND isnotnull(ss_customer_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dom#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#10, d_year#11, d_dom#12] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#10, d_year#11, d_dom#12] -Condition : ((((isnotnull(d_dom#12) AND (d_dom#12 >= 1)) AND (d_dom#12 <= 2)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#11, d_dom#12] - -(8) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(10) Project [codegen id : 5] -Output [8]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9] -Input [10]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9, d_date_sk#10] - -(11) Scan parquet default.store -Output [2]: [s_store_sk#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Midway,Fairview]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#14, s_city#15] - -(13) Filter [codegen id : 2] -Input [2]: [s_store_sk#14, s_city#15] -Condition : (s_city#15 IN (Midway,Fairview) AND isnotnull(s_store_sk#14)) - -(14) Project [codegen id : 2] -Output [1]: [s_store_sk#14] -Input [2]: [s_store_sk#14, s_city#15] - -(15) BroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#14] -Join condition: None - -(17) Project [codegen id : 5] -Output [7]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9] -Input [9]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9, s_store_sk#14] - -(18) Scan parquet default.household_demographics -Output [3]: [hd_demo_sk#17, hd_dep_count#18, hd_vehicle_count#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [3]: [hd_demo_sk#17, hd_dep_count#18, hd_vehicle_count#19] - -(20) Filter [codegen id : 3] -Input [3]: [hd_demo_sk#17, hd_dep_count#18, hd_vehicle_count#19] -Condition : (((hd_dep_count#18 = 4) OR (hd_vehicle_count#19 = 3)) AND isnotnull(hd_demo_sk#17)) - -(21) Project [codegen id : 3] -Output [1]: [hd_demo_sk#17] -Input [3]: [hd_demo_sk#17, hd_dep_count#18, hd_vehicle_count#19] - -(22) BroadcastExchange -Input [1]: [hd_demo_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#17] -Join condition: None - -(24) Project [codegen id : 5] -Output [6]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9] -Input [8]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9, hd_demo_sk#17] - -(25) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_city#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#21, ca_city#22] - -(27) Filter [codegen id : 4] -Input [2]: [ca_address_sk#21, ca_city#22] -Condition : (isnotnull(ca_address_sk#21) AND isnotnull(ca_city#22)) - -(28) BroadcastExchange -Input [2]: [ca_address_sk#21, ca_city#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] - -(29) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#4] -Right keys [1]: [ca_address_sk#21] -Join condition: None - -(30) Project [codegen id : 5] -Output [7]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9, ca_city#22] -Input [8]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9, ca_address_sk#21, ca_city#22] - -(31) HashAggregate [codegen id : 5] -Input [7]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_ext_sales_price#7, ss_ext_list_price#8, ss_ext_tax#9, ca_city#22] -Keys [4]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#22] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#7)), partial_sum(UnscaledValue(ss_ext_list_price#8)), partial_sum(UnscaledValue(ss_ext_tax#9))] -Aggregate Attributes [3]: [sum#24, sum#25, sum#26] -Results [7]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#22, sum#27, sum#28, sum#29] - -(32) Exchange -Input [7]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#22, sum#27, sum#28, sum#29] -Arguments: hashpartitioning(ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#22, 5), true, [id=#30] - -(33) HashAggregate [codegen id : 8] -Input [7]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#22, sum#27, sum#28, sum#29] -Keys [4]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#22] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#7)), sum(UnscaledValue(ss_ext_list_price#8)), sum(UnscaledValue(ss_ext_tax#9))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#7))#31, sum(UnscaledValue(ss_ext_list_price#8))#32, sum(UnscaledValue(ss_ext_tax#9))#33] -Results [6]: [ss_ticket_number#6, ss_customer_sk#2, ca_city#22 AS bought_city#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#31,17,2) AS extended_price#35, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#8))#32,17,2) AS list_price#36, MakeDecimal(sum(UnscaledValue(ss_ext_tax#9))#33,17,2) AS extended_tax#37] - -(34) Scan parquet default.customer -Output [4]: [c_customer_sk#38, c_current_addr_sk#39, c_first_name#40, c_last_name#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#38, c_current_addr_sk#39, c_first_name#40, c_last_name#41] - -(36) Filter [codegen id : 6] -Input [4]: [c_customer_sk#38, c_current_addr_sk#39, c_first_name#40, c_last_name#41] -Condition : (isnotnull(c_customer_sk#38) AND isnotnull(c_current_addr_sk#39)) - -(37) BroadcastExchange -Input [4]: [c_customer_sk#38, c_current_addr_sk#39, c_first_name#40, c_last_name#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#42] - -(38) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#38] -Join condition: None - -(39) Project [codegen id : 8] -Output [8]: [ss_ticket_number#6, bought_city#34, extended_price#35, list_price#36, extended_tax#37, c_current_addr_sk#39, c_first_name#40, c_last_name#41] -Input [10]: [ss_ticket_number#6, ss_customer_sk#2, bought_city#34, extended_price#35, list_price#36, extended_tax#37, c_customer_sk#38, c_current_addr_sk#39, c_first_name#40, c_last_name#41] - -(40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#21, ca_city#22] - -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#39] -Right keys [1]: [ca_address_sk#21] -Join condition: NOT (ca_city#22 = bought_city#34) - -(42) Project [codegen id : 8] -Output [8]: [c_last_name#41, c_first_name#40, ca_city#22, bought_city#34, ss_ticket_number#6, extended_price#35, extended_tax#37, list_price#36] -Input [10]: [ss_ticket_number#6, bought_city#34, extended_price#35, list_price#36, extended_tax#37, c_current_addr_sk#39, c_first_name#40, c_last_name#41, ca_address_sk#21, ca_city#22] - -(43) TakeOrderedAndProject -Input [8]: [c_last_name#41, c_first_name#40, ca_city#22, bought_city#34, ss_ticket_number#6, extended_price#35, extended_tax#37, list_price#36] -Arguments: 100, [c_last_name#41 ASC NULLS FIRST, ss_ticket_number#6 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, ca_city#22, bought_city#34, ss_ticket_number#6, extended_price#35, extended_tax#37, list_price#36] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q68/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q68/simplified.txt deleted file mode 100644 index 819e5504f..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q68/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (8) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - InputAdapter - Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_dom,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [s_store_sk] - Filter [s_city,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [hd_demo_sk] - Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [ca_address_sk,ca_city] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q69/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q69/explain.txt deleted file mode 100644 index b21892546..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q69/explain.txt +++ /dev/null @@ -1,274 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (32) - : : +- * BroadcastHashJoin LeftAnti BuildRight (31) - : : :- * BroadcastHashJoin LeftAnti BuildRight (23) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (19) - : : +- BroadcastExchange (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Filter (26) - : : : +- * ColumnarToRow (25) - : : : +- Scan parquet default.catalog_sales (24) - : : +- ReusedExchange (27) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.customer_address (33) - +- BroadcastExchange (43) - +- * Filter (42) - +- * ColumnarToRow (41) - +- Scan parquet default.customer_demographics (40) - - -(1) Scan parquet default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(3) Filter [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(4) Scan parquet default.store_sales -Output [2]: [ss_sold_date_sk#4, ss_customer_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_sold_date_sk#4, ss_customer_sk#5] - -(6) Filter [codegen id : 2] -Input [2]: [ss_sold_date_sk#4, ss_customer_sk#5] -Condition : isnotnull(ss_sold_date_sk#4) - -(7) Scan parquet default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] - -(9) Filter [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : (((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 >= 4)) AND (d_moy#8 <= 6)) AND isnotnull(d_date_sk#6)) - -(10) Project [codegen id : 1] -Output [1]: [d_date_sk#6] -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] - -(11) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] - -(12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join condition: None - -(13) Project [codegen id : 2] -Output [1]: [ss_customer_sk#5] -Input [3]: [ss_sold_date_sk#4, ss_customer_sk#5, d_date_sk#6] - -(14) BroadcastExchange -Input [1]: [ss_customer_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] - -(15) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] -Join condition: None - -(16) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 4] -Input [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] - -(18) Filter [codegen id : 4] -Input [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] -Condition : isnotnull(ws_sold_date_sk#11) - -(19) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#6] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#6] -Join condition: None - -(21) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12] -Input [3]: [ws_sold_date_sk#11, ws_bill_customer_sk#12, d_date_sk#6] - -(22) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#12] -Join condition: None - -(24) Scan parquet default.catalog_sales -Output [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 6] -Input [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] - -(26) Filter [codegen id : 6] -Input [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] -Condition : isnotnull(cs_sold_date_sk#14) - -(27) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#6] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#6] -Join condition: None - -(29) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#15] -Input [3]: [cs_sold_date_sk#14, cs_ship_customer_sk#15, d_date_sk#6] - -(30) BroadcastExchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#15] -Join condition: None - -(32) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(33) Scan parquet default.customer_address -Output [2]: [ca_address_sk#17, ca_state#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#17, ca_state#18] - -(35) Filter [codegen id : 7] -Input [2]: [ca_address_sk#17, ca_state#18] -Condition : (ca_state#18 IN (KY,GA,NM) AND isnotnull(ca_address_sk#17)) - -(36) Project [codegen id : 7] -Output [1]: [ca_address_sk#17] -Input [2]: [ca_address_sk#17, ca_state#18] - -(37) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#17] -Join condition: None - -(39) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#17] - -(40) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] - -(42) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Condition : isnotnull(cd_demo_sk#20) - -(43) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join condition: None - -(45) Project [codegen id : 9] -Output [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] - -(46) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Keys [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#27] -Results [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#28] - -(47) Exchange -Input [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#28] -Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, 5), true, [id=#29] - -(48) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#28] -Keys [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#30 AS cnt1#31, cd_purchase_estimate#24, count(1)#30 AS cnt2#32, cd_credit_rating#25, count(1)#30 AS cnt3#33] - -(49) TakeOrderedAndProject -Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33] -Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q69/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q69/simplified.txt deleted file mode 100644 index 9e34940da..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q69/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - InputAdapter - Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q7/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q7/explain.txt deleted file mode 100644 index 24546c68f..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q7/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * HashAggregate (33) - +- Exchange (32) - +- * HashAggregate (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.customer_demographics (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.date_dim (11) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.item (18) - +- BroadcastExchange (28) - +- * Project (27) - +- * Filter (26) - +- * ColumnarToRow (25) - +- Scan parquet default.promotion (24) - - -(1) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -Condition : (((isnotnull(ss_cdemo_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_promo_sk#4)) - -(4) Scan parquet default.customer_demographics -Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -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)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] - -(6) Filter [codegen id : 1] -Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) - -(7) Project [codegen id : 1] -Output [1]: [cd_demo_sk#9] -Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#9] -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, cd_demo_sk#9] - -(11) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#14, d_year#15] - -(13) Filter [codegen id : 2] -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(14) Project [codegen id : 2] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] - -(15) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#14] -Join condition: None - -(17) Project [codegen id : 5] -Output [6]: [ss_item_sk#2, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, d_date_sk#14] - -(18) Scan parquet default.item -Output [2]: [i_item_sk#17, i_item_id#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#17, i_item_id#18] - -(20) Filter [codegen id : 3] -Input [2]: [i_item_sk#17, i_item_id#18] -Condition : isnotnull(i_item_sk#17) - -(21) BroadcastExchange -Input [2]: [i_item_sk#17, i_item_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] - -(22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#17] -Join condition: None - -(23) Project [codegen id : 5] -Output [6]: [ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#18] -Input [8]: [ss_item_sk#2, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_sk#17, i_item_id#18] - -(24) Scan parquet default.promotion -Output [3]: [p_promo_sk#20, p_channel_email#21, p_channel_event#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [Or(EqualTo(p_channel_email,N),EqualTo(p_channel_event,N)), IsNotNull(p_promo_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 4] -Input [3]: [p_promo_sk#20, p_channel_email#21, p_channel_event#22] - -(26) Filter [codegen id : 4] -Input [3]: [p_promo_sk#20, p_channel_email#21, p_channel_event#22] -Condition : (((p_channel_email#21 = N) OR (p_channel_event#22 = N)) AND isnotnull(p_promo_sk#20)) - -(27) Project [codegen id : 4] -Output [1]: [p_promo_sk#20] -Input [3]: [p_promo_sk#20, p_channel_email#21, p_channel_event#22] - -(28) BroadcastExchange -Input [1]: [p_promo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] - -(29) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#20] -Join condition: None - -(30) Project [codegen id : 5] -Output [5]: [ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#18] -Input [7]: [ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#18, p_promo_sk#20] - -(31) HashAggregate [codegen id : 5] -Input [5]: [ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(cast(ss_quantity#5 as bigint)), partial_avg(UnscaledValue(ss_list_price#6)), partial_avg(UnscaledValue(ss_coupon_amt#8)), partial_avg(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [8]: [sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] -Results [9]: [i_item_id#18, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] - -(32) Exchange -Input [9]: [i_item_id#18, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -Arguments: hashpartitioning(i_item_id#18, 5), true, [id=#40] - -(33) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#18, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(cast(ss_quantity#5 as bigint)), avg(UnscaledValue(ss_list_price#6)), avg(UnscaledValue(ss_coupon_amt#8)), avg(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [4]: [avg(cast(ss_quantity#5 as bigint))#41, avg(UnscaledValue(ss_list_price#6))#42, avg(UnscaledValue(ss_coupon_amt#8))#43, avg(UnscaledValue(ss_sales_price#7))#44] -Results [5]: [i_item_id#18, avg(cast(ss_quantity#5 as bigint))#41 AS agg1#45, cast((avg(UnscaledValue(ss_list_price#6))#42 / 100.0) as decimal(11,6)) AS agg2#46, cast((avg(UnscaledValue(ss_coupon_amt#8))#43 / 100.0) as decimal(11,6)) AS agg3#47, cast((avg(UnscaledValue(ss_sales_price#7))#44 / 100.0) as decimal(11,6)) AS agg4#48] - -(34) TakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#45, agg2#46, agg3#47, agg4#48] -Arguments: 100, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#45, agg2#46, agg3#47, agg4#48] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q7/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q7/simplified.txt deleted file mode 100644 index 075c6d15c..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q7/simplified.txt +++ /dev/null @@ -1,50 +0,0 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_sold_date_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [cd_demo_sk] - Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [p_promo_sk] - Filter [p_channel_email,p_channel_event,p_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q70/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q70/explain.txt deleted file mode 100644 index 05b533aa6..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q70/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * Project (46) - +- Window (45) - +- * Sort (44) - +- Exchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Expand (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- BroadcastExchange (36) - +- * BroadcastHashJoin LeftSemi BuildRight (35) - :- * Filter (13) - : +- * ColumnarToRow (12) - : +- Scan parquet default.store (11) - +- BroadcastExchange (34) - +- * Project (33) - +- * Filter (32) - +- Window (31) - +- * Sort (30) - +- Exchange (29) - +- * HashAggregate (28) - +- Exchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet default.store_sales (14) - : +- BroadcastExchange (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet default.store (17) - +- ReusedExchange (23) - - -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 9] -Input [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] - -(3) Filter [codegen id : 9] -Input [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#2)) - -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_month_seq#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] -Condition : (((isnotnull(d_month_seq#5) AND (d_month_seq#5 >= 1200)) AND (d_month_seq#5 <= 1211)) AND isnotnull(d_date_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] - -(9) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(10) Project [codegen id : 9] -Output [2]: [ss_store_sk#2, ss_net_profit#3] -Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3, d_date_sk#4] - -(11) Scan parquet default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) Filter [codegen id : 8] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(14) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] - -(16) Filter [codegen id : 4] -Input [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_sold_date_sk#1)) - -(17) Scan parquet default.store -Output [2]: [s_store_sk#7, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#7, s_state#9] - -(19) Filter [codegen id : 2] -Input [2]: [s_store_sk#7, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(20) BroadcastExchange -Input [2]: [s_store_sk#7, s_state#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#7] -Join condition: None - -(22) Project [codegen id : 4] -Output [3]: [ss_sold_date_sk#1, ss_net_profit#3, s_state#9] -Input [5]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3, s_store_sk#7, s_state#9] - -(23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#4] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(25) Project [codegen id : 4] -Output [2]: [ss_net_profit#3, s_state#9] -Input [4]: [ss_sold_date_sk#1, ss_net_profit#3, s_state#9, d_date_sk#4] - -(26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#3, s_state#9] -Keys [1]: [s_state#9] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [sum#11] -Results [2]: [s_state#9, sum#12] - -(27) Exchange -Input [2]: [s_state#9, sum#12] -Arguments: hashpartitioning(s_state#9, 5), true, [id=#13] - -(28) HashAggregate [codegen id : 5] -Input [2]: [s_state#9, sum#12] -Keys [1]: [s_state#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#3))#14] -Results [3]: [s_state#9 AS s_state#15, s_state#9, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#14,17,2) AS _w2#16] - -(29) Exchange -Input [3]: [s_state#15, s_state#9, _w2#16] -Arguments: hashpartitioning(s_state#9, 5), true, [id=#17] - -(30) Sort [codegen id : 6] -Input [3]: [s_state#15, s_state#9, _w2#16] -Arguments: [s_state#9 ASC NULLS FIRST, _w2#16 DESC NULLS LAST], false, 0 - -(31) Window -Input [3]: [s_state#15, s_state#9, _w2#16] -Arguments: [rank(_w2#16) windowspecdefinition(s_state#9, _w2#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#18], [s_state#9], [_w2#16 DESC NULLS LAST] - -(32) Filter [codegen id : 7] -Input [4]: [s_state#15, s_state#9, _w2#16, ranking#18] -Condition : (isnotnull(ranking#18) AND (ranking#18 <= 5)) - -(33) Project [codegen id : 7] -Output [1]: [s_state#15] -Input [4]: [s_state#15, s_state#9, _w2#16, ranking#18] - -(34) BroadcastExchange -Input [1]: [s_state#15] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#19] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_state#9] -Right keys [1]: [s_state#15] -Join condition: None - -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#7] -Join condition: None - -(38) Project [codegen id : 9] -Output [3]: [ss_net_profit#3, s_state#9, s_county#8] -Input [5]: [ss_store_sk#2, ss_net_profit#3, s_store_sk#7, s_county#8, s_state#9] - -(39) Expand [codegen id : 9] -Input [3]: [ss_net_profit#3, s_state#9, s_county#8] -Arguments: [List(ss_net_profit#3, s_state#9, s_county#8, 0), List(ss_net_profit#3, s_state#9, null, 1), List(ss_net_profit#3, null, null, 3)], [ss_net_profit#3, s_state#21, s_county#22, spark_grouping_id#23] - -(40) HashAggregate [codegen id : 9] -Input [4]: [ss_net_profit#3, s_state#21, s_county#22, spark_grouping_id#23] -Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [sum#24] -Results [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] - -(41) Exchange -Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -Arguments: hashpartitioning(s_state#21, s_county#22, spark_grouping_id#23, 5), true, [id=#26] - -(42) HashAggregate [codegen id : 10] -Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] -Functions [1]: [sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#3))#27] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#27,17,2) AS total_sum#28, s_state#21, s_county#22, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS lochierarchy#29, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS _w1#30, CASE WHEN (cast(cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) as int) = 0) THEN s_state#21 END AS _w2#31, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#27,17,2) AS _w3#32] - -(43) Exchange -Input [7]: [total_sum#28, s_state#21, s_county#22, lochierarchy#29, _w1#30, _w2#31, _w3#32] -Arguments: hashpartitioning(_w1#30, _w2#31, 5), true, [id=#33] - -(44) Sort [codegen id : 11] -Input [7]: [total_sum#28, s_state#21, s_county#22, lochierarchy#29, _w1#30, _w2#31, _w3#32] -Arguments: [_w1#30 ASC NULLS FIRST, _w2#31 ASC NULLS FIRST, _w3#32 DESC NULLS LAST], false, 0 - -(45) Window -Input [7]: [total_sum#28, s_state#21, s_county#22, lochierarchy#29, _w1#30, _w2#31, _w3#32] -Arguments: [rank(_w3#32) windowspecdefinition(_w1#30, _w2#31, _w3#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#30, _w2#31], [_w3#32 DESC NULLS LAST] - -(46) Project [codegen id : 12] -Output [5]: [total_sum#28, s_state#21, s_county#22, lochierarchy#29, rank_within_parent#34] -Input [8]: [total_sum#28, s_state#21, s_county#22, lochierarchy#29, _w1#30, _w2#31, _w3#32, rank_within_parent#34] - -(47) TakeOrderedAndProject -Input [5]: [total_sum#28, s_state#21, s_county#22, lochierarchy#29, rank_within_parent#34] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (cast(lochierarchy#29 as int) = 0) THEN s_state#21 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#28, s_state#21, s_county#22, lochierarchy#29, rank_within_parent#34] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q70/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q70/simplified.txt deleted file mode 100644 index 5d01429ad..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q70/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (12) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w3,_w1,_w2] - WholeStageCodegen (11) - Sort [_w1,_w2,_w3] - InputAdapter - Exchange [_w1,_w2] #1 - WholeStageCodegen (10) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w1,_w2,_w3,sum] - InputAdapter - Exchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (9) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) - BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w2,s_state] - WholeStageCodegen (6) - Sort [s_state,_w2] - InputAdapter - Exchange [s_state] #6 - WholeStageCodegen (5) - HashAggregate [s_state,sum] [sum(UnscaledValue(ss_net_profit)),s_state,_w2,sum] - InputAdapter - Exchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_net_profit,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q71/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q71/explain.txt deleted file mode 100644 index 9471377a1..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q71/explain.txt +++ /dev/null @@ -1,232 +0,0 @@ -== Physical Plan == -* Sort (42) -+- Exchange (41) - +- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildLeft (29) - : :- BroadcastExchange (5) - : : +- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.item (1) - : +- Union (28) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet default.web_sales (6) - : : +- BroadcastExchange (13) - : : +- * Project (12) - : : +- * Filter (11) - : : +- * ColumnarToRow (10) - : : +- Scan parquet default.date_dim (9) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * Filter (18) - : : : +- * ColumnarToRow (17) - : : : +- Scan parquet default.catalog_sales (16) - : : +- ReusedExchange (19) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.store_sales (22) - : +- ReusedExchange (25) - +- BroadcastExchange (35) - +- * Project (34) - +- * Filter (33) - +- * ColumnarToRow (32) - +- Scan parquet default.time_dim (31) - - -(1) Scan parquet default.item -Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] - -(3) Filter [codegen id : 1] -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) - -(4) Project [codegen id : 1] -Output [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] - -(5) BroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#5] - -(6) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#6, ws_sold_time_sk#7, ws_item_sk#8, ws_ext_sales_price#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] -ReadSchema: struct - -(7) ColumnarToRow [codegen id : 3] -Input [4]: [ws_sold_date_sk#6, ws_sold_time_sk#7, ws_item_sk#8, ws_ext_sales_price#9] - -(8) Filter [codegen id : 3] -Input [4]: [ws_sold_date_sk#6, ws_sold_time_sk#7, ws_item_sk#8, ws_ext_sales_price#9] -Condition : ((isnotnull(ws_sold_date_sk#6) AND isnotnull(ws_item_sk#8)) AND isnotnull(ws_sold_time_sk#7)) - -(9) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_moy#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] - -(11) Filter [codegen id : 2] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11)) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) - -(12) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] - -(13) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(15) Project [codegen id : 3] -Output [3]: [ws_ext_sales_price#9 AS ext_price#14, ws_item_sk#8 AS sold_item_sk#15, ws_sold_time_sk#7 AS time_sk#16] -Input [5]: [ws_sold_date_sk#6, ws_sold_time_sk#7, ws_item_sk#8, ws_ext_sales_price#9, d_date_sk#10] - -(16) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#17, cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 5] -Input [4]: [cs_sold_date_sk#17, cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(18) Filter [codegen id : 5] -Input [4]: [cs_sold_date_sk#17, cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Condition : ((isnotnull(cs_sold_date_sk#17) AND isnotnull(cs_item_sk#19)) AND isnotnull(cs_sold_time_sk#18)) - -(19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#10] - -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(21) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#20 AS ext_price#21, cs_item_sk#19 AS sold_item_sk#22, cs_sold_time_sk#18 AS time_sk#23] -Input [5]: [cs_sold_date_sk#17, cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, d_date_sk#10] - -(22) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#24, ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] -ReadSchema: struct - -(23) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_date_sk#24, ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27] - -(24) Filter [codegen id : 7] -Input [4]: [ss_sold_date_sk#24, ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27] -Condition : ((isnotnull(ss_sold_date_sk#24) AND isnotnull(ss_item_sk#26)) AND isnotnull(ss_sold_time_sk#25)) - -(25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#10] - -(26) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#24] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(27) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#27 AS ext_price#28, ss_item_sk#26 AS sold_item_sk#29, ss_sold_time_sk#25 AS time_sk#30] -Input [5]: [ss_sold_date_sk#24, ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, d_date_sk#10] - -(28) Union - -(29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [sold_item_sk#15] -Join condition: None - -(30) Project [codegen id : 9] -Output [4]: [i_brand_id#2, i_brand#3, ext_price#14, time_sk#16] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#14, sold_item_sk#15, time_sk#16] - -(31) Scan parquet default.time_dim -Output [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -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)] -ReadSchema: struct - -(32) ColumnarToRow [codegen id : 8] -Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] - -(33) Filter [codegen id : 8] -Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -Condition : (((t_meal_time#34 = breakfast) OR (t_meal_time#34 = dinner)) AND isnotnull(t_time_sk#31)) - -(34) Project [codegen id : 8] -Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] - -(35) BroadcastExchange -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] - -(36) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [time_sk#16] -Right keys [1]: [t_time_sk#31] -Join condition: None - -(37) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#14, t_hour#32, t_minute#33] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#14, time_sk#16, t_time_sk#31, t_hour#32, t_minute#33] - -(38) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#14, t_hour#32, t_minute#33] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#32, t_minute#33] -Functions [1]: [partial_sum(UnscaledValue(ext_price#14))] -Aggregate Attributes [1]: [sum#36] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#32, t_minute#33, sum#37] - -(39) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#32, t_minute#33, sum#37] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#32, t_minute#33, 5), true, [id=#38] - -(40) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#32, t_minute#33, sum#37] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#32, t_minute#33] -Functions [1]: [sum(UnscaledValue(ext_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#14))#39] -Results [5]: [i_brand_id#2 AS brand_id#40, i_brand#3 AS brand#41, t_hour#32, t_minute#33, MakeDecimal(sum(UnscaledValue(ext_price#14))#39,17,2) AS ext_price#42] - -(41) Exchange -Input [5]: [brand_id#40, brand#41, t_hour#32, t_minute#33, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), true, [id=#43] - -(42) Sort [codegen id : 11] -Input [5]: [brand_id#40, brand#41, t_hour#32, t_minute#33, ext_price#42] -Arguments: [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q71/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q71/simplified.txt deleted file mode 100644 index 7fb18bbd6..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q71/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (11) - Sort [ext_price,brand_id] - InputAdapter - Exchange [ext_price,brand_id] #1 - WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - InputAdapter - Exchange [i_brand,i_brand_id,t_hour,t_minute] #2 - WholeStageCodegen (9) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_item_sk,ws_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_sold_time_sk,ws_item_sk,ws_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_item_sk,cs_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_sold_time_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_item_sk,ss_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_sold_time_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Project [t_time_sk,t_hour,t_minute] - Filter [t_meal_time,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q72/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q72/explain.txt deleted file mode 100644 index 5ca28c3e3..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q72/explain.txt +++ /dev/null @@ -1,391 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin LeftOuter BuildRight (65) - :- * Project (60) - : +- * BroadcastHashJoin LeftOuter BuildRight (59) - : :- * Project (54) - : : +- * BroadcastHashJoin Inner BuildRight (53) - : : :- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * Filter (3) - : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * Filter (6) - : : : : : : : : : +- * ColumnarToRow (5) - : : : : : : : : : +- Scan parquet default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * Filter (12) - : : : : : : : : +- * ColumnarToRow (11) - : : : : : : : : +- Scan parquet default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * Filter (18) - : : : : : : : +- * ColumnarToRow (17) - : : : : : : : +- Scan parquet default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * Project (25) - : : : : : : +- * Filter (24) - : : : : : : +- * ColumnarToRow (23) - : : : : : : +- Scan parquet default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * Project (32) - : : : : : +- * Filter (31) - : : : : : +- * ColumnarToRow (30) - : : : : : +- Scan parquet default.household_demographics (29) - : : : : +- BroadcastExchange (40) - : : : : +- * Project (39) - : : : : +- * Filter (38) - : : : : +- * ColumnarToRow (37) - : : : : +- Scan parquet default.date_dim (36) - : : : +- BroadcastExchange (46) - : : : +- * Filter (45) - : : : +- * ColumnarToRow (44) - : : : +- Scan parquet default.date_dim (43) - : : +- BroadcastExchange (52) - : : +- * Filter (51) - : : +- * ColumnarToRow (50) - : : +- Scan parquet default.date_dim (49) - : +- BroadcastExchange (58) - : +- * Filter (57) - : +- * ColumnarToRow (56) - : +- Scan parquet default.promotion (55) - +- BroadcastExchange (64) - +- * Filter (63) - +- * ColumnarToRow (62) - +- Scan parquet default.catalog_returns (61) - - -(1) Scan parquet default.catalog_sales -Output [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 11] -Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] - -(3) Filter [codegen id : 11] -Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] -Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnotnull(cs_bill_cdemo_sk#3)) AND isnotnull(cs_bill_hdemo_sk#4)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_ship_date_sk#2)) - -(4) Scan parquet default.inventory -Output [4]: [inv_date_sk#9, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [inv_date_sk#9, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12] - -(6) Filter [codegen id : 1] -Input [4]: [inv_date_sk#9, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12] -Condition : (((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) AND isnotnull(inv_date_sk#9)) - -(7) BroadcastExchange -Input [4]: [inv_date_sk#9, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#13] - -(8) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#5] -Right keys [1]: [inv_item_sk#10] -Join condition: (inv_quantity_on_hand#12 < cs_quantity#8) - -(9) Project [codegen id : 11] -Output [9]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, inv_warehouse_sk#11] -Input [12]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, inv_date_sk#9, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12] - -(10) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] - -(12) Filter [codegen id : 2] -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] - -(14) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join condition: None - -(15) Project [codegen id : 11] -Output [9]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, w_warehouse_name#15] -Input [11]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, inv_warehouse_sk#11, w_warehouse_sk#14, w_warehouse_name#15] - -(16) Scan parquet default.item -Output [2]: [i_item_sk#17, i_item_desc#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#17, i_item_desc#18] - -(18) Filter [codegen id : 3] -Input [2]: [i_item_sk#17, i_item_desc#18] -Condition : isnotnull(i_item_sk#17) - -(19) BroadcastExchange -Input [2]: [i_item_sk#17, i_item_desc#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] - -(20) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#5] -Right keys [1]: [i_item_sk#17] -Join condition: None - -(21) Project [codegen id : 11] -Output [10]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, w_warehouse_name#15, i_item_desc#18] -Input [11]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, w_warehouse_name#15, i_item_sk#17, i_item_desc#18] - -(22) Scan parquet default.customer_demographics -Output [2]: [cd_demo_sk#20, cd_marital_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,D), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(23) ColumnarToRow [codegen id : 4] -Input [2]: [cd_demo_sk#20, cd_marital_status#21] - -(24) Filter [codegen id : 4] -Input [2]: [cd_demo_sk#20, cd_marital_status#21] -Condition : ((isnotnull(cd_marital_status#21) AND (cd_marital_status#21 = D)) AND isnotnull(cd_demo_sk#20)) - -(25) Project [codegen id : 4] -Output [1]: [cd_demo_sk#20] -Input [2]: [cd_demo_sk#20, cd_marital_status#21] - -(26) BroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] - -(27) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#20] -Join condition: None - -(28) Project [codegen id : 11] -Output [9]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, w_warehouse_name#15, i_item_desc#18] -Input [11]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, w_warehouse_name#15, i_item_desc#18, cd_demo_sk#20] - -(29) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#23, hd_buy_potential#24] -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)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 5] -Input [2]: [hd_demo_sk#23, hd_buy_potential#24] - -(31) Filter [codegen id : 5] -Input [2]: [hd_demo_sk#23, hd_buy_potential#24] -Condition : ((isnotnull(hd_buy_potential#24) AND (hd_buy_potential#24 = >10000)) AND isnotnull(hd_demo_sk#23)) - -(32) Project [codegen id : 5] -Output [1]: [hd_demo_sk#23] -Input [2]: [hd_demo_sk#23, hd_buy_potential#24] - -(33) BroadcastExchange -Input [1]: [hd_demo_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#23] -Join condition: None - -(35) Project [codegen id : 11] -Output [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, w_warehouse_name#15, i_item_desc#18] -Input [10]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, w_warehouse_name#15, i_item_desc#18, hd_demo_sk#23] - -(36) Scan parquet default.date_dim -Output [4]: [d_date_sk#26, d_date#27, d_week_seq#28, d_year#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(37) ColumnarToRow [codegen id : 6] -Input [4]: [d_date_sk#26, d_date#27, d_week_seq#28, d_year#29] - -(38) Filter [codegen id : 6] -Input [4]: [d_date_sk#26, d_date#27, d_week_seq#28, d_year#29] -Condition : ((((isnotnull(d_year#29) AND (d_year#29 = 1999)) AND isnotnull(d_date_sk#26)) AND isnotnull(d_week_seq#28)) AND isnotnull(d_date#27)) - -(39) Project [codegen id : 6] -Output [3]: [d_date_sk#26, d_date#27, d_week_seq#28] -Input [4]: [d_date_sk#26, d_date#27, d_week_seq#28, d_year#29] - -(40) BroadcastExchange -Input [3]: [d_date_sk#26, d_date#27, d_week_seq#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] - -(41) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#26] -Join condition: None - -(42) Project [codegen id : 11] -Output [9]: [cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, w_warehouse_name#15, i_item_desc#18, d_date#27, d_week_seq#28] -Input [11]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, w_warehouse_name#15, i_item_desc#18, d_date_sk#26, d_date#27, d_week_seq#28] - -(43) Scan parquet default.date_dim -Output [2]: [d_date_sk#31, d_week_seq#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#31, d_week_seq#32] - -(45) Filter [codegen id : 7] -Input [2]: [d_date_sk#31, d_week_seq#32] -Condition : (isnotnull(d_week_seq#32) AND isnotnull(d_date_sk#31)) - -(46) BroadcastExchange -Input [2]: [d_date_sk#31, d_week_seq#32] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [id=#33] - -(47) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [d_week_seq#28, inv_date_sk#9] -Right keys [2]: [d_week_seq#32, d_date_sk#31] -Join condition: None - -(48) Project [codegen id : 11] -Output [8]: [cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#15, i_item_desc#18, d_date#27, d_week_seq#28] -Input [11]: [cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, inv_date_sk#9, w_warehouse_name#15, i_item_desc#18, d_date#27, d_week_seq#28, d_date_sk#31, d_week_seq#32] - -(49) Scan parquet default.date_dim -Output [2]: [d_date_sk#34, d_date#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#34, d_date#35] - -(51) Filter [codegen id : 8] -Input [2]: [d_date_sk#34, d_date#35] -Condition : (isnotnull(d_date#35) AND isnotnull(d_date_sk#34)) - -(52) BroadcastExchange -Input [2]: [d_date_sk#34, d_date#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] - -(53) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_ship_date_sk#2] -Right keys [1]: [d_date_sk#34] -Join condition: (d_date#35 > d_date#27 + 5 days) - -(54) Project [codegen id : 11] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#15, i_item_desc#18, d_week_seq#28] -Input [10]: [cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#15, i_item_desc#18, d_date#27, d_week_seq#28, d_date_sk#34, d_date#35] - -(55) Scan parquet default.promotion -Output [1]: [p_promo_sk#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(56) ColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#37] - -(57) Filter [codegen id : 9] -Input [1]: [p_promo_sk#37] -Condition : isnotnull(p_promo_sk#37) - -(58) BroadcastExchange -Input [1]: [p_promo_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] - -(59) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_promo_sk#6] -Right keys [1]: [p_promo_sk#37] -Join condition: None - -(60) Project [codegen id : 11] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#15, i_item_desc#18, d_week_seq#28] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#15, i_item_desc#18, d_week_seq#28, p_promo_sk#37] - -(61) Scan parquet default.catalog_returns -Output [2]: [cr_item_sk#39, cr_order_number#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(62) ColumnarToRow [codegen id : 10] -Input [2]: [cr_item_sk#39, cr_order_number#40] - -(63) Filter [codegen id : 10] -Input [2]: [cr_item_sk#39, cr_order_number#40] -Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40)) - -(64) BroadcastExchange -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [id=#41] - -(65) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [cs_item_sk#5, cs_order_number#7] -Right keys [2]: [cr_item_sk#39, cr_order_number#40] -Join condition: None - -(66) Project [codegen id : 11] -Output [3]: [w_warehouse_name#15, i_item_desc#18, d_week_seq#28] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#15, i_item_desc#18, d_week_seq#28, cr_item_sk#39, cr_order_number#40] - -(67) HashAggregate [codegen id : 11] -Input [3]: [w_warehouse_name#15, i_item_desc#18, d_week_seq#28] -Keys [3]: [i_item_desc#18, w_warehouse_name#15, d_week_seq#28] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#42] -Results [4]: [i_item_desc#18, w_warehouse_name#15, d_week_seq#28, count#43] - -(68) Exchange -Input [4]: [i_item_desc#18, w_warehouse_name#15, d_week_seq#28, count#43] -Arguments: hashpartitioning(i_item_desc#18, w_warehouse_name#15, d_week_seq#28, 5), true, [id=#44] - -(69) HashAggregate [codegen id : 12] -Input [4]: [i_item_desc#18, w_warehouse_name#15, d_week_seq#28, count#43] -Keys [3]: [i_item_desc#18, w_warehouse_name#15, d_week_seq#28] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#45] -Results [6]: [i_item_desc#18, w_warehouse_name#15, d_week_seq#28, count(1)#45 AS no_promo#46, count(1)#45 AS promo#47, count(1)#45 AS total_cnt#48] - -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#18, w_warehouse_name#15, d_week_seq#28, no_promo#46, promo#47, total_cnt#48] -Arguments: 100, [total_cnt#48 DESC NULLS LAST, i_item_desc#18 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#28 ASC NULLS FIRST], [i_item_desc#18, w_warehouse_name#15, d_week_seq#28, no_promo#46, promo#47, total_cnt#48] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q72/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q72/simplified.txt deleted file mode 100644 index 25f03cbac..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q72/simplified.txt +++ /dev/null @@ -1,104 +0,0 @@ -TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (12) - HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] - InputAdapter - Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (11) - HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] - Project [w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,inv_warehouse_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Filter [p_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.promotion [p_promo_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (10) - Filter [cr_item_sk,cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q73/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q73/explain.txt deleted file mode 100644 index f4565c3ed..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q73/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -* Sort (36) -+- Exchange (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Filter (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.store (11) - : +- BroadcastExchange (22) - : +- * Project (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.household_demographics (18) - +- BroadcastExchange (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.customer (29) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5] -Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_hdemo_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_dom#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#7, d_dom#8] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#7, d_dom#8] -Condition : ((((isnotnull(d_dom#8) AND (d_dom#8 >= 1)) AND (d_dom#8 <= 2)) AND d_year#7 IN (1999,2000,2001)) AND isnotnull(d_date_sk#6)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#6] -Input [3]: [d_date_sk#6, d_year#7, d_dom#8] - -(8) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#6] -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5] -Input [6]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5, d_date_sk#6] - -(11) Scan parquet default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_county, [Williamson County,Franklin Parish,Bronx County,Orange County]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#10, s_county#11] - -(13) Filter [codegen id : 2] -Input [2]: [s_store_sk#10, s_county#11] -Condition : (s_county#11 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#10)) - -(14) Project [codegen id : 2] -Output [1]: [s_store_sk#10] -Input [2]: [s_store_sk#10, s_county#11] - -(15) BroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#10] -Join condition: None - -(17) Project [codegen id : 4] -Output [3]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_ticket_number#5] -Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5, s_store_sk#10] - -(18) Scan parquet default.household_demographics -Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_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] -Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] - -(20) Filter [codegen id : 3] -Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = >10000) OR (hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND (CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND isnotnull(hd_demo_sk#13)) - -(21) Project [codegen id : 3] -Output [1]: [hd_demo_sk#13] -Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] - -(22) BroadcastExchange -Input [1]: [hd_demo_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#13] -Join condition: None - -(24) Project [codegen id : 4] -Output [2]: [ss_customer_sk#2, ss_ticket_number#5] -Input [4]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_ticket_number#5, hd_demo_sk#13] - -(25) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#2, ss_ticket_number#5] -Keys [2]: [ss_ticket_number#5, ss_customer_sk#2] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] - -(26) Exchange -Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, [id=#20] - -(27) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] -Keys [2]: [ss_ticket_number#5, ss_customer_sk#2] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#21] -Results [3]: [ss_ticket_number#5, ss_customer_sk#2, count(1)#21 AS cnt#22] - -(28) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22] -Condition : ((cnt#22 >= 1) AND (cnt#22 <= 5)) - -(29) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27] - -(31) Filter [codegen id : 5] -Input [5]: [c_customer_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27] -Condition : isnotnull(c_customer_sk#23) - -(32) BroadcastExchange -Input [5]: [c_customer_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] - -(33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#23] -Join condition: None - -(34) Project [codegen id : 6] -Output [6]: [c_last_name#26, c_first_name#25, c_salutation#24, c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22] -Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, c_customer_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27] - -(35) Exchange -Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22] -Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), true, [id=#29] - -(36) Sort [codegen id : 7] -Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22] -Arguments: [cnt#22 DESC NULLS LAST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q73/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q73/simplified.txt deleted file mode 100644 index 46b724156..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q73/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (7) - Sort [cnt] - InputAdapter - Exchange [cnt] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - InputAdapter - Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_dom,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [s_store_sk] - Filter [s_county,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q74/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q74/explain.txt deleted file mode 100644 index 4c9bad96e..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q74/explain.txt +++ /dev/null @@ -1,410 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Filter (19) - : : : +- * HashAggregate (18) - : : : +- Exchange (17) - : : : +- * HashAggregate (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_sales (4) - : : : +- BroadcastExchange (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (35) - : : +- * HashAggregate (34) - : : +- Exchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * Filter (22) - : : : : +- * ColumnarToRow (21) - : : : : +- Scan parquet default.customer (20) - : : : +- ReusedExchange (23) - : : +- BroadcastExchange (29) - : : +- * Filter (28) - : : +- * ColumnarToRow (27) - : : +- Scan parquet default.date_dim (26) - : +- BroadcastExchange (54) - : +- * Project (53) - : +- * Filter (52) - : +- * HashAggregate (51) - : +- Exchange (50) - : +- * HashAggregate (49) - : +- * Project (48) - : +- * BroadcastHashJoin Inner BuildRight (47) - : :- * Project (45) - : : +- * BroadcastHashJoin Inner BuildRight (44) - : : :- * Filter (39) - : : : +- * ColumnarToRow (38) - : : : +- Scan parquet default.customer (37) - : : +- BroadcastExchange (43) - : : +- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.web_sales (40) - : +- ReusedExchange (46) - +- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Filter (59) - : : +- * ColumnarToRow (58) - : : +- Scan parquet default.customer (57) - : +- ReusedExchange (60) - +- ReusedExchange (63) - - -(1) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] - -(3) Filter [codegen id : 3] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(4) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#5, ss_customer_sk#6, ss_net_paid#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [ss_sold_date_sk#5, ss_customer_sk#6, ss_net_paid#7] - -(6) Filter [codegen id : 1] -Input [3]: [ss_sold_date_sk#5, ss_customer_sk#6, ss_net_paid#7] -Condition : (isnotnull(ss_customer_sk#6) AND isnotnull(ss_sold_date_sk#5)) - -(7) BroadcastExchange -Input [3]: [ss_sold_date_sk#5, ss_customer_sk#6, ss_net_paid#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#8] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#6] -Join condition: None - -(9) Project [codegen id : 3] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_sold_date_sk#5, ss_net_paid#7] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_sold_date_sk#5, ss_customer_sk#6, ss_net_paid#7] - -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#9, d_year#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#9, d_year#10] - -(12) Filter [codegen id : 2] -Input [2]: [d_date_sk#9, d_year#10] -Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) - -(13) BroadcastExchange -Input [2]: [d_date_sk#9, d_year#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] - -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#9] -Join condition: None - -(15) Project [codegen id : 3] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#7, d_year#10] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_sold_date_sk#5, ss_net_paid#7, d_date_sk#9, d_year#10] - -(16) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#7, d_year#10] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#7))] -Aggregate Attributes [1]: [sum#12] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#13] - -(17) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#13] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), true, [id=#14] - -(18) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#13] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] -Functions [1]: [sum(UnscaledValue(ss_net_paid#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#7))#15] -Results [2]: [c_customer_id#2 AS customer_id#16, MakeDecimal(sum(UnscaledValue(ss_net_paid#7))#15,17,2) AS year_total#17] - -(19) Filter [codegen id : 16] -Input [2]: [customer_id#16, year_total#17] -Condition : (isnotnull(year_total#17) AND (year_total#17 > 0.00)) - -(20) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] - -(22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(23) ReusedExchange [Reuses operator id: 7] -Output [3]: [ss_sold_date_sk#5, ss_customer_sk#6, ss_net_paid#7] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#6] -Join condition: None - -(25) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_sold_date_sk#5, ss_net_paid#7] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_sold_date_sk#5, ss_customer_sk#6, ss_net_paid#7] - -(26) Scan parquet default.date_dim -Output [2]: [d_date_sk#9, d_year#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#9, d_year#10] - -(28) Filter [codegen id : 5] -Input [2]: [d_date_sk#9, d_year#10] -Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2002)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) - -(29) BroadcastExchange -Input [2]: [d_date_sk#9, d_year#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#9] -Join condition: None - -(31) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#7, d_year#10] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_sold_date_sk#5, ss_net_paid#7, d_date_sk#9, d_year#10] - -(32) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#7, d_year#10] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#7))] -Aggregate Attributes [1]: [sum#19] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#20] - -(33) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#20] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), true, [id=#21] - -(34) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#20] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] -Functions [1]: [sum(UnscaledValue(ss_net_paid#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#7))#22] -Results [4]: [c_customer_id#2 AS customer_id#23, c_first_name#3 AS customer_first_name#24, c_last_name#4 AS customer_last_name#25, MakeDecimal(sum(UnscaledValue(ss_net_paid#7))#22,17,2) AS year_total#26] - -(35) BroadcastExchange -Input [4]: [customer_id#23, customer_first_name#24, customer_last_name#25, year_total#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#27] - -(36) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#23] -Join condition: None - -(37) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] - -(39) Filter [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(40) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#28, ws_bill_customer_sk#29, ws_net_paid#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 8] -Input [3]: [ws_sold_date_sk#28, ws_bill_customer_sk#29, ws_net_paid#30] - -(42) Filter [codegen id : 8] -Input [3]: [ws_sold_date_sk#28, ws_bill_customer_sk#29, ws_net_paid#30] -Condition : (isnotnull(ws_bill_customer_sk#29) AND isnotnull(ws_sold_date_sk#28)) - -(43) BroadcastExchange -Input [3]: [ws_sold_date_sk#28, ws_bill_customer_sk#29, ws_net_paid#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#31] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#29] -Join condition: None - -(45) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_sold_date_sk#28, ws_net_paid#30] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_sold_date_sk#28, ws_bill_customer_sk#29, ws_net_paid#30] - -(46) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#9, d_year#10] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#28] -Right keys [1]: [d_date_sk#9] -Join condition: None - -(48) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#30, d_year#10] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_sold_date_sk#28, ws_net_paid#30, d_date_sk#9, d_year#10] - -(49) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#30, d_year#10] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#30))] -Aggregate Attributes [1]: [sum#32] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#33] - -(50) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#33] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), true, [id=#34] - -(51) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#33] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#30))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#30))#35] -Results [2]: [c_customer_id#2 AS customer_id#36, MakeDecimal(sum(UnscaledValue(ws_net_paid#30))#35,17,2) AS year_total#37] - -(52) Filter [codegen id : 11] -Input [2]: [customer_id#36, year_total#37] -Condition : (isnotnull(year_total#37) AND (year_total#37 > 0.00)) - -(53) Project [codegen id : 11] -Output [2]: [customer_id#36 AS customer_id#38, year_total#37 AS year_total#39] -Input [2]: [customer_id#36, year_total#37] - -(54) BroadcastExchange -Input [2]: [customer_id#38, year_total#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] - -(55) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#38] -Join condition: None - -(56) Project [codegen id : 16] -Output [7]: [customer_id#16, year_total#17, customer_id#23, customer_first_name#24, customer_last_name#25, year_total#26, year_total#39] -Input [8]: [customer_id#16, year_total#17, customer_id#23, customer_first_name#24, customer_last_name#25, year_total#26, customer_id#38, year_total#39] - -(57) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] -ReadSchema: struct - -(58) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] - -(59) Filter [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) - -(60) ReusedExchange [Reuses operator id: 43] -Output [3]: [ws_sold_date_sk#28, ws_bill_customer_sk#29, ws_net_paid#30] - -(61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#29] -Join condition: None - -(62) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_sold_date_sk#28, ws_net_paid#30] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_sold_date_sk#28, ws_bill_customer_sk#29, ws_net_paid#30] - -(63) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#9, d_year#10] - -(64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#28] -Right keys [1]: [d_date_sk#9] -Join condition: None - -(65) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#30, d_year#10] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_sold_date_sk#28, ws_net_paid#30, d_date_sk#9, d_year#10] - -(66) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#30, d_year#10] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#30))] -Aggregate Attributes [1]: [sum#41] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#42] - -(67) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#42] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), true, [id=#43] - -(68) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#42] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#30))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#30))#44] -Results [2]: [c_customer_id#2 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#30))#44,17,2) AS year_total#46] - -(69) BroadcastExchange -Input [2]: [customer_id#45, year_total#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] - -(70) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#45] -Join condition: (CASE WHEN (year_total#39 > 0.00) THEN CheckOverflow((promote_precision(year_total#46) / promote_precision(year_total#39)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#17 > 0.00) THEN CheckOverflow((promote_precision(year_total#26) / promote_precision(year_total#17)), DecimalType(37,20), true) ELSE null END) - -(71) Project [codegen id : 16] -Output [3]: [customer_id#23, customer_first_name#24, customer_last_name#25] -Input [9]: [customer_id#16, year_total#17, customer_id#23, customer_first_name#24, customer_last_name#25, year_total#26, year_total#39, customer_id#45, year_total#46] - -(72) TakeOrderedAndProject -Input [3]: [customer_id#23, customer_first_name#24, customer_last_name#25] -Arguments: 100, [customer_id#23 ASC NULLS FIRST, customer_id#23 ASC NULLS FIRST, customer_id#23 ASC NULLS FIRST], [customer_id#23, customer_first_name#24, customer_last_name#25] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q74/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q74/simplified.txt deleted file mode 100644 index add2d43fc..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q74/simplified.txt +++ /dev/null @@ -1,107 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_sold_date_sk,ss_net_paid] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_net_paid] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_sold_date_sk,ss_net_paid] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - ReusedExchange [ss_sold_date_sk,ss_customer_sk,ss_net_paid] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - Project [customer_id,year_total] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #8 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_sold_date_sk,ws_net_paid] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_net_paid] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_sold_date_sk,ws_net_paid] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - ReusedExchange [ws_sold_date_sk,ws_bill_customer_sk,ws_net_paid] #9 - InputAdapter - ReusedExchange [d_date_sk,d_year] #6 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q75/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q75/explain.txt deleted file mode 100644 index 292a44930..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q75/explain.txt +++ /dev/null @@ -1,647 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (117) -+- * Project (116) - +- * BroadcastHashJoin Inner BuildRight (115) - :- * HashAggregate (63) - : +- Exchange (62) - : +- * HashAggregate (61) - : +- * HashAggregate (60) - : +- Exchange (59) - : +- * HashAggregate (58) - : +- Union (57) - : :- * HashAggregate (41) - : : +- Exchange (40) - : : +- * HashAggregate (39) - : : +- Union (38) - : : :- * Project (22) - : : : +- * BroadcastHashJoin LeftOuter BuildRight (21) - : : : :- * Project (16) - : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * Project (7) - : : : : : +- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.item (4) - : : : : +- BroadcastExchange (14) - : : : : +- * Filter (13) - : : : : +- * ColumnarToRow (12) - : : : : +- Scan parquet default.date_dim (11) - : : : +- BroadcastExchange (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_returns (17) - : : +- * Project (37) - : : +- * BroadcastHashJoin LeftOuter BuildRight (36) - : : :- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * Filter (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- Scan parquet default.store_sales (23) - : : : : +- ReusedExchange (26) - : : : +- ReusedExchange (29) - : : +- BroadcastExchange (35) - : : +- * Filter (34) - : : +- * ColumnarToRow (33) - : : +- Scan parquet default.store_returns (32) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (47) - : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : :- * Filter (44) - : : : : +- * ColumnarToRow (43) - : : : : +- Scan parquet default.web_sales (42) - : : : +- ReusedExchange (45) - : : +- ReusedExchange (48) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * ColumnarToRow (52) - : +- Scan parquet default.web_returns (51) - +- BroadcastExchange (114) - +- * HashAggregate (113) - +- Exchange (112) - +- * HashAggregate (111) - +- * HashAggregate (110) - +- Exchange (109) - +- * HashAggregate (108) - +- Union (107) - :- * HashAggregate (94) - : +- Exchange (93) - : +- * HashAggregate (92) - : +- Union (91) - : :- * Project (78) - : : +- * BroadcastHashJoin LeftOuter BuildRight (77) - : : :- * Project (75) - : : : +- * BroadcastHashJoin Inner BuildRight (74) - : : : :- * Project (69) - : : : : +- * BroadcastHashJoin Inner BuildRight (68) - : : : : :- * Filter (66) - : : : : : +- * ColumnarToRow (65) - : : : : : +- Scan parquet default.catalog_sales (64) - : : : : +- ReusedExchange (67) - : : : +- BroadcastExchange (73) - : : : +- * Filter (72) - : : : +- * ColumnarToRow (71) - : : : +- Scan parquet default.date_dim (70) - : : +- ReusedExchange (76) - : +- * Project (90) - : +- * BroadcastHashJoin LeftOuter BuildRight (89) - : :- * Project (87) - : : +- * BroadcastHashJoin Inner BuildRight (86) - : : :- * Project (84) - : : : +- * BroadcastHashJoin Inner BuildRight (83) - : : : :- * Filter (81) - : : : : +- * ColumnarToRow (80) - : : : : +- Scan parquet default.store_sales (79) - : : : +- ReusedExchange (82) - : : +- ReusedExchange (85) - : +- ReusedExchange (88) - +- * Project (106) - +- * BroadcastHashJoin LeftOuter BuildRight (105) - :- * Project (103) - : +- * BroadcastHashJoin Inner BuildRight (102) - : :- * Project (100) - : : +- * BroadcastHashJoin Inner BuildRight (99) - : : :- * Filter (97) - : : : +- * ColumnarToRow (96) - : : : +- Scan parquet default.web_sales (95) - : : +- ReusedExchange (98) - : +- ReusedExchange (101) - +- ReusedExchange (104) - - -(1) Scan parquet default.catalog_sales -Output [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] - -(3) Filter [codegen id : 4] -Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) - -(4) Scan parquet default.item -Output [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] - -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] -Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_manufact_id#11)) - -(7) Project [codegen id : 1] -Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] -Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] - -(8) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#6] -Join condition: None - -(10) Project [codegen id : 4] -Output [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] -Input [10]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] - -(11) Scan parquet default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#13, d_year#14] - -(13) Filter [codegen id : 2] -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(14) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(16) Project [codegen id : 4] -Output [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] -Input [11]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] - -(17) Scan parquet default.catalog_returns -Output [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] - -(19) Filter [codegen id : 3] -Input [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] -Condition : (isnotnull(cr_order_number#17) AND isnotnull(cr_item_sk#16)) - -(20) BroadcastExchange -Input [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [id=#20] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [cs_order_number#3, cs_item_sk#2] -Right keys [2]: [cr_order_number#17, cr_item_sk#16] -Join condition: None - -(22) Project [codegen id : 4] -Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (cs_quantity#4 - coalesce(cr_return_quantity#18, 0)) AS sales_cnt#21, CheckOverflow((promote_precision(cast(cs_ext_sales_price#5 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#19, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#22] -Input [13]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] - -(23) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 8] -Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] - -(25) Filter [codegen id : 8] -Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] -Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_date_sk#23)) - -(26) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] - -(27) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#24] -Right keys [1]: [i_item_sk#6] -Join condition: None - -(28) Project [codegen id : 8] -Output [9]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] -Input [10]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] - -(29) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#13, d_year#14] - -(30) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#23] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(31) Project [codegen id : 8] -Output [9]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] -Input [11]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] - -(32) Scan parquet default.store_returns -Output [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 7] -Input [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] - -(34) Filter [codegen id : 7] -Input [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] -Condition : (isnotnull(sr_ticket_number#29) AND isnotnull(sr_item_sk#28)) - -(35) BroadcastExchange -Input [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#32] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [cast(ss_ticket_number#25 as bigint), cast(ss_item_sk#24 as bigint)] -Right keys [2]: [sr_ticket_number#29, sr_item_sk#28] -Join condition: None - -(37) Project [codegen id : 8] -Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (ss_quantity#26 - coalesce(sr_return_quantity#30, 0)) AS sales_cnt#33, CheckOverflow((promote_precision(cast(ss_ext_sales_price#27 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#31, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#34] -Input [13]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] - -(38) Union - -(39) HashAggregate [codegen id : 9] -Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] - -(40) Exchange -Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22, 5), true, [id=#35] - -(41) HashAggregate [codegen id : 10] -Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] - -(42) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 14] -Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] - -(44) Filter [codegen id : 14] -Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] -Condition : (isnotnull(ws_item_sk#37) AND isnotnull(ws_sold_date_sk#36)) - -(45) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] - -(46) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#6] -Join condition: None - -(47) Project [codegen id : 14] -Output [9]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] -Input [10]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] - -(48) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#13, d_year#14] - -(49) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(50) Project [codegen id : 14] -Output [9]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14] -Input [11]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_date_sk#13, d_year#14] - -(51) Scan parquet default.web_returns -Output [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(52) ColumnarToRow [codegen id : 13] -Input [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] - -(53) Filter [codegen id : 13] -Input [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] -Condition : (isnotnull(wr_order_number#42) AND isnotnull(wr_item_sk#41)) - -(54) BroadcastExchange -Input [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#45] - -(55) BroadcastHashJoin [codegen id : 14] -Left keys [2]: [cast(ws_order_number#38 as bigint), cast(ws_item_sk#37 as bigint)] -Right keys [2]: [wr_order_number#42, wr_item_sk#41] -Join condition: None - -(56) Project [codegen id : 14] -Output [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, (ws_quantity#39 - coalesce(wr_return_quantity#43, 0)) AS sales_cnt#46, CheckOverflow((promote_precision(cast(ws_ext_sales_price#40 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#44, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#47] -Input [13]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, d_year#14, wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] - -(57) Union - -(58) HashAggregate [codegen id : 15] -Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] - -(59) Exchange -Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22, 5), true, [id=#48] - -(60) HashAggregate [codegen id : 16] -Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Keys [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] - -(61) HashAggregate [codegen id : 16] -Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#21, sales_amt#22] -Keys [5]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] -Functions [2]: [partial_sum(cast(sales_cnt#21 as bigint)), partial_sum(UnscaledValue(sales_amt#22))] -Aggregate Attributes [2]: [sum#49, sum#50] -Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#51, sum#52] - -(62) Exchange -Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#51, sum#52] -Arguments: hashpartitioning(d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, 5), true, [id=#53] - -(63) HashAggregate [codegen id : 34] -Input [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum#51, sum#52] -Keys [5]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] -Functions [2]: [sum(cast(sales_cnt#21 as bigint)), sum(UnscaledValue(sales_amt#22))] -Aggregate Attributes [2]: [sum(cast(sales_cnt#21 as bigint))#54, sum(UnscaledValue(sales_amt#22))#55] -Results [7]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sum(cast(sales_cnt#21 as bigint))#54 AS sales_cnt#56, MakeDecimal(sum(UnscaledValue(sales_amt#22))#55,18,2) AS sales_amt#57] - -(64) Scan parquet default.catalog_sales -Output [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(65) ColumnarToRow [codegen id : 20] -Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] - -(66) Filter [codegen id : 20] -Input [5]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) - -(67) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] - -(68) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#58] -Join condition: None - -(69) Project [codegen id : 20] -Output [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] -Input [10]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] - -(70) Scan parquet default.date_dim -Output [2]: [d_date_sk#63, d_year#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 18] -Input [2]: [d_date_sk#63, d_year#64] - -(72) Filter [codegen id : 18] -Input [2]: [d_date_sk#63, d_year#64] -Condition : ((isnotnull(d_year#64) AND (d_year#64 = 2001)) AND isnotnull(d_date_sk#63)) - -(73) BroadcastExchange -Input [2]: [d_date_sk#63, d_year#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] - -(74) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#63] -Join condition: None - -(75) Project [codegen id : 20] -Output [9]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64] -Input [11]: [cs_sold_date_sk#1, cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_date_sk#63, d_year#64] - -(76) ReusedExchange [Reuses operator id: 20] -Output [4]: [cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] - -(77) BroadcastHashJoin [codegen id : 20] -Left keys [2]: [cs_order_number#3, cs_item_sk#2] -Right keys [2]: [cr_order_number#17, cr_item_sk#16] -Join condition: None - -(78) Project [codegen id : 20] -Output [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, (cs_quantity#4 - coalesce(cr_return_quantity#18, 0)) AS sales_cnt#21, CheckOverflow((promote_precision(cast(cs_ext_sales_price#5 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#19, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#22] -Input [13]: [cs_item_sk#2, cs_order_number#3, cs_quantity#4, cs_ext_sales_price#5, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64, cr_item_sk#16, cr_order_number#17, cr_return_quantity#18, cr_return_amount#19] - -(79) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(80) ColumnarToRow [codegen id : 24] -Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] - -(81) Filter [codegen id : 24] -Input [5]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27] -Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_date_sk#23)) - -(82) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] - -(83) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_item_sk#24] -Right keys [1]: [i_item_sk#58] -Join condition: None - -(84) Project [codegen id : 24] -Output [9]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] -Input [10]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] - -(85) ReusedExchange [Reuses operator id: 73] -Output [2]: [d_date_sk#63, d_year#64] - -(86) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_sold_date_sk#23] -Right keys [1]: [d_date_sk#63] -Join condition: None - -(87) Project [codegen id : 24] -Output [9]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64] -Input [11]: [ss_sold_date_sk#23, ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_date_sk#63, d_year#64] - -(88) ReusedExchange [Reuses operator id: 35] -Output [4]: [sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] - -(89) BroadcastHashJoin [codegen id : 24] -Left keys [2]: [cast(ss_ticket_number#25 as bigint), cast(ss_item_sk#24 as bigint)] -Right keys [2]: [sr_ticket_number#29, sr_item_sk#28] -Join condition: None - -(90) Project [codegen id : 24] -Output [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, (ss_quantity#26 - coalesce(sr_return_quantity#30, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ss_ext_sales_price#27 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#31, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] -Input [13]: [ss_item_sk#24, ss_ticket_number#25, ss_quantity#26, ss_ext_sales_price#27, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64, sr_item_sk#28, sr_ticket_number#29, sr_return_quantity#30, sr_return_amt#31] - -(91) Union - -(92) HashAggregate [codegen id : 25] -Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] - -(93) Exchange -Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Arguments: hashpartitioning(d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22, 5), true, [id=#68] - -(94) HashAggregate [codegen id : 26] -Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] - -(95) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(96) ColumnarToRow [codegen id : 30] -Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] - -(97) Filter [codegen id : 30] -Input [5]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40] -Condition : (isnotnull(ws_item_sk#37) AND isnotnull(ws_sold_date_sk#36)) - -(98) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] - -(99) BroadcastHashJoin [codegen id : 30] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#58] -Join condition: None - -(100) Project [codegen id : 30] -Output [9]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] -Input [10]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] - -(101) ReusedExchange [Reuses operator id: 73] -Output [2]: [d_date_sk#63, d_year#64] - -(102) BroadcastHashJoin [codegen id : 30] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#63] -Join condition: None - -(103) Project [codegen id : 30] -Output [9]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64] -Input [11]: [ws_sold_date_sk#36, ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_date_sk#63, d_year#64] - -(104) ReusedExchange [Reuses operator id: 54] -Output [4]: [wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] - -(105) BroadcastHashJoin [codegen id : 30] -Left keys [2]: [cast(ws_order_number#38 as bigint), cast(ws_item_sk#37 as bigint)] -Right keys [2]: [wr_order_number#42, wr_item_sk#41] -Join condition: None - -(106) Project [codegen id : 30] -Output [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, (ws_quantity#39 - coalesce(wr_return_quantity#43, 0)) AS sales_cnt#69, CheckOverflow((promote_precision(cast(ws_ext_sales_price#40 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#44, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#70] -Input [13]: [ws_item_sk#37, ws_order_number#38, ws_quantity#39, ws_ext_sales_price#40, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, d_year#64, wr_item_sk#41, wr_order_number#42, wr_return_quantity#43, wr_return_amt#44] - -(107) Union - -(108) HashAggregate [codegen id : 31] -Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] - -(109) Exchange -Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Arguments: hashpartitioning(d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22, 5), true, [id=#71] - -(110) HashAggregate [codegen id : 32] -Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Keys [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] - -(111) HashAggregate [codegen id : 32] -Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#21, sales_amt#22] -Keys [5]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] -Functions [2]: [partial_sum(cast(sales_cnt#21 as bigint)), partial_sum(UnscaledValue(sales_amt#22))] -Aggregate Attributes [2]: [sum#72, sum#73] -Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum#74, sum#75] - -(112) Exchange -Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum#74, sum#75] -Arguments: hashpartitioning(d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, 5), true, [id=#76] - -(113) HashAggregate [codegen id : 33] -Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum#74, sum#75] -Keys [5]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] -Functions [2]: [sum(cast(sales_cnt#21 as bigint)), sum(UnscaledValue(sales_amt#22))] -Aggregate Attributes [2]: [sum(cast(sales_cnt#21 as bigint))#77, sum(UnscaledValue(sales_amt#22))#78] -Results [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sum(cast(sales_cnt#21 as bigint))#77 AS sales_cnt#79, MakeDecimal(sum(UnscaledValue(sales_amt#22))#78,18,2) AS sales_amt#80] - -(114) BroadcastExchange -Input [7]: [d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#79, sales_amt#80] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true], input[4, int, true]),false), [id=#81] - -(115) BroadcastHashJoin [codegen id : 34] -Left keys [4]: [i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] -Right keys [4]: [i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#56 as decimal(17,2))) / promote_precision(cast(sales_cnt#79 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) - -(116) Project [codegen id : 34] -Output [10]: [d_year#64 AS prev_year#82, d_year#14 AS year#83, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#79 AS prev_yr_cnt#84, sales_cnt#56 AS curr_yr_cnt#85, (sales_cnt#56 - sales_cnt#79) AS sales_cnt_diff#86, CheckOverflow((promote_precision(cast(sales_amt#57 as decimal(19,2))) - promote_precision(cast(sales_amt#80 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#87] -Input [14]: [d_year#14, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, sales_cnt#56, sales_amt#57, d_year#64, i_brand_id#59, i_class_id#60, i_category_id#61, i_manufact_id#62, sales_cnt#79, sales_amt#80] - -(117) TakeOrderedAndProject -Input [10]: [prev_year#82, year#83, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, prev_yr_cnt#84, curr_yr_cnt#85, sales_cnt_diff#86, sales_amt_diff#87] -Arguments: 100, [sales_cnt_diff#86 ASC NULLS FIRST], [prev_year#82, year#83, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11, prev_yr_cnt#84, curr_yr_cnt#85, sales_cnt_diff#86, sales_amt_diff#87] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q75/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q75/simplified.txt deleted file mode 100644 index 298a06b87..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q75/simplified.txt +++ /dev/null @@ -1,180 +0,0 @@ -TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] - WholeStageCodegen (34) - Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (16) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #2 - WholeStageCodegen (15) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - WholeStageCodegen (10) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - WholeStageCodegen (9) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - WholeStageCodegen (4) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Filter [cr_order_number,cr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - WholeStageCodegen (8) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - WholeStageCodegen (14) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (13) - Filter [wr_order_number,wr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (33) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #10 - WholeStageCodegen (32) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #11 - WholeStageCodegen (31) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - WholeStageCodegen (26) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #12 - WholeStageCodegen (25) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - WholeStageCodegen (20) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (18) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #6 - WholeStageCodegen (24) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year] #13 - InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #7 - WholeStageCodegen (30) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year] #13 - InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #8 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q76/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q76/explain.txt deleted file mode 100644 index 917e2b028..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q76/explain.txt +++ /dev/null @@ -1,209 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (38) -+- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- Union (34) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.item (4) - : +- BroadcastExchange (13) - : +- * Filter (12) - : +- * ColumnarToRow (11) - : +- Scan parquet default.date_dim (10) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * Filter (18) - : : : +- * ColumnarToRow (17) - : : : +- Scan parquet default.web_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Filter (27) - : : +- * ColumnarToRow (26) - : : +- Scan parquet default.catalog_sales (25) - : +- ReusedExchange (28) - +- ReusedExchange (31) - - -(1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_ext_sales_price#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_ext_sales_price#4] - -(3) Filter [codegen id : 3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_ext_sales_price#4] -Condition : ((isnull(ss_store_sk#3) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_sold_date_sk#1)) - -(4) Scan parquet default.item -Output [2]: [i_item_sk#5, i_category#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_category#6] - -(6) Filter [codegen id : 1] -Input [2]: [i_item_sk#5, i_category#6] -Condition : isnotnull(i_item_sk#5) - -(7) BroadcastExchange -Input [2]: [i_item_sk#5, i_category#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(9) Project [codegen id : 3] -Output [4]: [ss_sold_date_sk#1, ss_store_sk#3, ss_ext_sales_price#4, i_category#6] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_ext_sales_price#4, i_item_sk#5, i_category#6] - -(10) Scan parquet default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] - -(12) Filter [codegen id : 2] -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : isnotnull(d_date_sk#8) - -(13) BroadcastExchange -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] - -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(15) Project [codegen id : 3] -Output [6]: [store AS channel#12, ss_store_sk#3 AS col_name#13, d_year#9, d_qoy#10, i_category#6, ss_ext_sales_price#4 AS ext_sales_price#14] -Input [7]: [ss_sold_date_sk#1, ss_store_sk#3, ss_ext_sales_price#4, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] - -(16) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#15, ws_item_sk#16, ws_ship_customer_sk#17, ws_ext_sales_price#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 6] -Input [4]: [ws_sold_date_sk#15, ws_item_sk#16, ws_ship_customer_sk#17, ws_ext_sales_price#18] - -(18) Filter [codegen id : 6] -Input [4]: [ws_sold_date_sk#15, ws_item_sk#16, ws_ship_customer_sk#17, ws_ext_sales_price#18] -Condition : ((isnull(ws_ship_customer_sk#17) AND isnotnull(ws_item_sk#16)) AND isnotnull(ws_sold_date_sk#15)) - -(19) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_category#6] - -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#16] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(21) Project [codegen id : 6] -Output [4]: [ws_sold_date_sk#15, ws_ship_customer_sk#17, ws_ext_sales_price#18, i_category#6] -Input [6]: [ws_sold_date_sk#15, ws_item_sk#16, ws_ship_customer_sk#17, ws_ext_sales_price#18, i_item_sk#5, i_category#6] - -(22) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(24) Project [codegen id : 6] -Output [6]: [web AS channel#19, ws_ship_customer_sk#17 AS col_name#20, d_year#9, d_qoy#10, i_category#6, ws_ext_sales_price#18 AS ext_sales_price#21] -Input [7]: [ws_sold_date_sk#15, ws_ship_customer_sk#17, ws_ext_sales_price#18, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] - -(25) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#22, cs_ship_addr_sk#23, cs_item_sk#24, cs_ext_sales_price#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 9] -Input [4]: [cs_sold_date_sk#22, cs_ship_addr_sk#23, cs_item_sk#24, cs_ext_sales_price#25] - -(27) Filter [codegen id : 9] -Input [4]: [cs_sold_date_sk#22, cs_ship_addr_sk#23, cs_item_sk#24, cs_ext_sales_price#25] -Condition : ((isnull(cs_ship_addr_sk#23) AND isnotnull(cs_item_sk#24)) AND isnotnull(cs_sold_date_sk#22)) - -(28) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_category#6] - -(29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#24] -Right keys [1]: [i_item_sk#5] -Join condition: None - -(30) Project [codegen id : 9] -Output [4]: [cs_sold_date_sk#22, cs_ship_addr_sk#23, cs_ext_sales_price#25, i_category#6] -Input [6]: [cs_sold_date_sk#22, cs_ship_addr_sk#23, cs_item_sk#24, cs_ext_sales_price#25, i_item_sk#5, i_category#6] - -(31) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#22] -Right keys [1]: [d_date_sk#8] -Join condition: None - -(33) Project [codegen id : 9] -Output [6]: [catalog AS channel#26, cs_ship_addr_sk#23 AS col_name#27, d_year#9, d_qoy#10, i_category#6, cs_ext_sales_price#25 AS ext_sales_price#28] -Input [7]: [cs_sold_date_sk#22, cs_ship_addr_sk#23, cs_ext_sales_price#25, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] - -(34) Union - -(35) HashAggregate [codegen id : 10] -Input [6]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, ext_sales_price#14] -Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] -Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#29, sum#30] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] - -(36) Exchange -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), true, [id=#33] - -(37) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] -Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] -Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#34, sum(UnscaledValue(ext_sales_price#14))#35] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#34 AS sales_cnt#36, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#35,17,2) AS sales_amt#37] - -(38) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q76/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q76/simplified.txt deleted file mode 100644 index f01916baa..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q76/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - WholeStageCodegen (11) - HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] - InputAdapter - Exchange [channel,col_name,d_year,d_qoy,i_category] #1 - WholeStageCodegen (10) - HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] - InputAdapter - Union - WholeStageCodegen (3) - Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_store_sk,ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - WholeStageCodegen (6) - Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_ship_customer_sk,ws_ext_sales_price,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_ship_customer_sk,ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price] - InputAdapter - ReusedExchange [i_item_sk,i_category] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - WholeStageCodegen (9) - Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_addr_sk,cs_ext_sales_price,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_ship_addr_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - ReusedExchange [i_item_sk,i_category] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q77/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q77/explain.txt deleted file mode 100644 index c232055ba..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q77/explain.txt +++ /dev/null @@ -1,520 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (91) -+- * HashAggregate (90) - +- Exchange (89) - +- * HashAggregate (88) - +- * Expand (87) - +- Union (86) - :- * Project (34) - : +- * BroadcastHashJoin LeftOuter BuildRight (33) - : :- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.store (11) - : +- BroadcastExchange (32) - : +- * HashAggregate (31) - : +- Exchange (30) - : +- * HashAggregate (29) - : +- * Project (28) - : +- * BroadcastHashJoin Inner BuildRight (27) - : :- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Filter (22) - : : : +- * ColumnarToRow (21) - : : : +- Scan parquet default.store_returns (20) - : : +- ReusedExchange (23) - : +- ReusedExchange (26) - :- * Project (55) - : +- BroadcastNestedLoopJoin Inner BuildLeft (54) - : :- BroadcastExchange (44) - : : +- * HashAggregate (43) - : : +- Exchange (42) - : : +- * HashAggregate (41) - : : +- * Project (40) - : : +- * BroadcastHashJoin Inner BuildRight (39) - : : :- * Filter (37) - : : : +- * ColumnarToRow (36) - : : : +- Scan parquet default.catalog_sales (35) - : : +- ReusedExchange (38) - : +- * HashAggregate (53) - : +- Exchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Filter (47) - : : +- * ColumnarToRow (46) - : : +- Scan parquet default.catalog_returns (45) - : +- ReusedExchange (48) - +- * Project (85) - +- * BroadcastHashJoin LeftOuter BuildRight (84) - :- * HashAggregate (70) - : +- Exchange (69) - : +- * HashAggregate (68) - : +- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Filter (58) - : : : +- * ColumnarToRow (57) - : : : +- Scan parquet default.web_sales (56) - : : +- ReusedExchange (59) - : +- BroadcastExchange (65) - : +- * Filter (64) - : +- * ColumnarToRow (63) - : +- Scan parquet default.web_page (62) - +- BroadcastExchange (83) - +- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * Filter (73) - : : +- * ColumnarToRow (72) - : : +- Scan parquet default.web_returns (71) - : +- ReusedExchange (74) - +- ReusedExchange (77) - - -(1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(3) Filter [codegen id : 3] -Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#2)) - -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -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 >= 11172)) AND (d_date#6 <= 11202)) AND isnotnull(d_date_sk#5)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] - -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Input [5]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, d_date_sk#5] - -(11) Scan parquet default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] - -(13) Filter [codegen id : 2] -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(14) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#9] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#8] -Join condition: None - -(16) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#3, ss_net_profit#4, s_store_sk#8] -Input [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, s_store_sk#8] - -(17) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#3, ss_net_profit#4, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#3)), partial_sum(UnscaledValue(ss_net_profit#4))] -Aggregate Attributes [2]: [sum#10, sum#11] -Results [3]: [s_store_sk#8, sum#12, sum#13] - -(18) Exchange -Input [3]: [s_store_sk#8, sum#12, sum#13] -Arguments: hashpartitioning(s_store_sk#8, 5), true, [id=#14] - -(19) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#8, sum#12, sum#13] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#3)), sum(UnscaledValue(ss_net_profit#4))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#3))#15, sum(UnscaledValue(ss_net_profit#4))#16] -Results [3]: [s_store_sk#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS sales#17, MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#16,17,2) AS profit#18] - -(20) Scan parquet default.store_returns -Output [4]: [sr_returned_date_sk#19, sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_returned_date_sk), IsNotNull(sr_store_sk)] -ReadSchema: struct - -(21) ColumnarToRow [codegen id : 6] -Input [4]: [sr_returned_date_sk#19, sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] - -(22) Filter [codegen id : 6] -Input [4]: [sr_returned_date_sk#19, sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Condition : (isnotnull(sr_returned_date_sk#19) AND isnotnull(sr_store_sk#20)) - -(23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [cast(d_date_sk#5 as bigint)] -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_returned_date_sk#19, sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, d_date_sk#5] - -(26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#23] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#23 as bigint)] -Join condition: None - -(28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#23] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#23] - -(29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#23] -Keys [1]: [s_store_sk#23] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#24, sum#25] -Results [3]: [s_store_sk#23, sum#26, sum#27] - -(30) Exchange -Input [3]: [s_store_sk#23, sum#26, sum#27] -Arguments: hashpartitioning(s_store_sk#23, 5), true, [id=#28] - -(31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#23, sum#26, sum#27] -Keys [1]: [s_store_sk#23] -Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#29, sum(UnscaledValue(sr_net_loss#22))#30] -Results [3]: [s_store_sk#23, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#29,17,2) AS returns#31, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#30,17,2) AS profit_loss#32] - -(32) BroadcastExchange -Input [3]: [s_store_sk#23, returns#31, profit_loss#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] - -(33) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#8] -Right keys [1]: [s_store_sk#23] -Join condition: None - -(34) Project [codegen id : 8] -Output [5]: [sales#17, coalesce(returns#31, 0.00) AS returns#34, CheckOverflow((promote_precision(cast(profit#18 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#32, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#35, store channel AS channel#36, s_store_sk#8 AS id#37] -Input [6]: [s_store_sk#8, sales#17, profit#18, s_store_sk#23, returns#31, profit_loss#32] - -(35) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#38, cs_call_center_sk#39, cs_ext_sales_price#40, cs_net_profit#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_sold_date_sk#38, cs_call_center_sk#39, cs_ext_sales_price#40, cs_net_profit#41] - -(37) Filter [codegen id : 10] -Input [4]: [cs_sold_date_sk#38, cs_call_center_sk#39, cs_ext_sales_price#40, cs_net_profit#41] -Condition : isnotnull(cs_sold_date_sk#38) - -(38) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#38] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(40) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#39, cs_ext_sales_price#40, cs_net_profit#41] -Input [5]: [cs_sold_date_sk#38, cs_call_center_sk#39, cs_ext_sales_price#40, cs_net_profit#41, d_date_sk#5] - -(41) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#39, cs_ext_sales_price#40, cs_net_profit#41] -Keys [1]: [cs_call_center_sk#39] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(UnscaledValue(cs_net_profit#41))] -Aggregate Attributes [2]: [sum#42, sum#43] -Results [3]: [cs_call_center_sk#39, sum#44, sum#45] - -(42) Exchange -Input [3]: [cs_call_center_sk#39, sum#44, sum#45] -Arguments: hashpartitioning(cs_call_center_sk#39, 5), true, [id=#46] - -(43) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#39, sum#44, sum#45] -Keys [1]: [cs_call_center_sk#39] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(UnscaledValue(cs_net_profit#41))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#40))#47, sum(UnscaledValue(cs_net_profit#41))#48] -Results [3]: [cs_call_center_sk#39, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#47,17,2) AS sales#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#41))#48,17,2) AS profit#50] - -(44) BroadcastExchange -Input [3]: [cs_call_center_sk#39, sales#49, profit#50] -Arguments: IdentityBroadcastMode, [id=#51] - -(45) Scan parquet default.catalog_returns -Output [3]: [cr_returned_date_sk#52, cr_return_amount#53, cr_net_loss#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 13] -Input [3]: [cr_returned_date_sk#52, cr_return_amount#53, cr_net_loss#54] - -(47) Filter [codegen id : 13] -Input [3]: [cr_returned_date_sk#52, cr_return_amount#53, cr_net_loss#54] -Condition : isnotnull(cr_returned_date_sk#52) - -(48) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#52] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(50) Project [codegen id : 13] -Output [2]: [cr_return_amount#53, cr_net_loss#54] -Input [4]: [cr_returned_date_sk#52, cr_return_amount#53, cr_net_loss#54, d_date_sk#5] - -(51) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#53, cr_net_loss#54] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [2]: [sum#57, sum#58] - -(52) Exchange -Input [2]: [sum#57, sum#58] -Arguments: SinglePartition, true, [id=#59] - -(53) HashAggregate [codegen id : 14] -Input [2]: [sum#57, sum#58] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#60, sum(UnscaledValue(cr_net_loss#54))#61] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#60,17,2) AS returns#62, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#61,17,2) AS profit_loss#63] - -(54) BroadcastNestedLoopJoin -Join condition: None - -(55) Project [codegen id : 15] -Output [5]: [sales#49, returns#62, CheckOverflow((promote_precision(cast(profit#50 as decimal(18,2))) - promote_precision(cast(profit_loss#63 as decimal(18,2)))), DecimalType(18,2), true) AS profit#64, catalog channel AS channel#65, cs_call_center_sk#39 AS id#66] -Input [5]: [cs_call_center_sk#39, sales#49, profit#50, returns#62, profit_loss#63] - -(56) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#67, ws_web_page_sk#68, ws_ext_sales_price#69, ws_net_profit#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(57) ColumnarToRow [codegen id : 18] -Input [4]: [ws_sold_date_sk#67, ws_web_page_sk#68, ws_ext_sales_price#69, ws_net_profit#70] - -(58) Filter [codegen id : 18] -Input [4]: [ws_sold_date_sk#67, ws_web_page_sk#68, ws_ext_sales_price#69, ws_net_profit#70] -Condition : (isnotnull(ws_sold_date_sk#67) AND isnotnull(ws_web_page_sk#68)) - -(59) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(60) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#67] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(61) Project [codegen id : 18] -Output [3]: [ws_web_page_sk#68, ws_ext_sales_price#69, ws_net_profit#70] -Input [5]: [ws_sold_date_sk#67, ws_web_page_sk#68, ws_ext_sales_price#69, ws_net_profit#70, d_date_sk#5] - -(62) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#71] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(63) ColumnarToRow [codegen id : 17] -Input [1]: [wp_web_page_sk#71] - -(64) Filter [codegen id : 17] -Input [1]: [wp_web_page_sk#71] -Condition : isnotnull(wp_web_page_sk#71) - -(65) BroadcastExchange -Input [1]: [wp_web_page_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] - -(66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_web_page_sk#68] -Right keys [1]: [wp_web_page_sk#71] -Join condition: None - -(67) Project [codegen id : 18] -Output [3]: [ws_ext_sales_price#69, ws_net_profit#70, wp_web_page_sk#71] -Input [4]: [ws_web_page_sk#68, ws_ext_sales_price#69, ws_net_profit#70, wp_web_page_sk#71] - -(68) HashAggregate [codegen id : 18] -Input [3]: [ws_ext_sales_price#69, ws_net_profit#70, wp_web_page_sk#71] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#69)), partial_sum(UnscaledValue(ws_net_profit#70))] -Aggregate Attributes [2]: [sum#73, sum#74] -Results [3]: [wp_web_page_sk#71, sum#75, sum#76] - -(69) Exchange -Input [3]: [wp_web_page_sk#71, sum#75, sum#76] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), true, [id=#77] - -(70) HashAggregate [codegen id : 23] -Input [3]: [wp_web_page_sk#71, sum#75, sum#76] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#69)), sum(UnscaledValue(ws_net_profit#70))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#69))#78, sum(UnscaledValue(ws_net_profit#70))#79] -Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#69))#78,17,2) AS sales#80, MakeDecimal(sum(UnscaledValue(ws_net_profit#70))#79,17,2) AS profit#81] - -(71) Scan parquet default.web_returns -Output [4]: [wr_returned_date_sk#82, wr_web_page_sk#83, wr_return_amt#84, wr_net_loss#85] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_returned_date_sk), IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(72) ColumnarToRow [codegen id : 21] -Input [4]: [wr_returned_date_sk#82, wr_web_page_sk#83, wr_return_amt#84, wr_net_loss#85] - -(73) Filter [codegen id : 21] -Input [4]: [wr_returned_date_sk#82, wr_web_page_sk#83, wr_return_amt#84, wr_net_loss#85] -Condition : (isnotnull(wr_returned_date_sk#82) AND isnotnull(wr_web_page_sk#83)) - -(74) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(75) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wr_returned_date_sk#82] -Right keys [1]: [cast(d_date_sk#5 as bigint)] -Join condition: None - -(76) Project [codegen id : 21] -Output [3]: [wr_web_page_sk#83, wr_return_amt#84, wr_net_loss#85] -Input [5]: [wr_returned_date_sk#82, wr_web_page_sk#83, wr_return_amt#84, wr_net_loss#85, d_date_sk#5] - -(77) ReusedExchange [Reuses operator id: 65] -Output [1]: [wp_web_page_sk#86] - -(78) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wr_web_page_sk#83] -Right keys [1]: [cast(wp_web_page_sk#86 as bigint)] -Join condition: None - -(79) Project [codegen id : 21] -Output [3]: [wr_return_amt#84, wr_net_loss#85, wp_web_page_sk#86] -Input [4]: [wr_web_page_sk#83, wr_return_amt#84, wr_net_loss#85, wp_web_page_sk#86] - -(80) HashAggregate [codegen id : 21] -Input [3]: [wr_return_amt#84, wr_net_loss#85, wp_web_page_sk#86] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#84)), partial_sum(UnscaledValue(wr_net_loss#85))] -Aggregate Attributes [2]: [sum#87, sum#88] -Results [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(81) Exchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), true, [id=#91] - -(82) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [sum(UnscaledValue(wr_return_amt#84)), sum(UnscaledValue(wr_net_loss#85))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#84))#92, sum(UnscaledValue(wr_net_loss#85))#93] -Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#84))#92,17,2) AS returns#94, MakeDecimal(sum(UnscaledValue(wr_net_loss#85))#93,17,2) AS profit_loss#95] - -(83) BroadcastExchange -Input [3]: [wp_web_page_sk#86, returns#94, profit_loss#95] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#96] - -(84) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [wp_web_page_sk#71] -Right keys [1]: [wp_web_page_sk#86] -Join condition: None - -(85) Project [codegen id : 23] -Output [5]: [sales#80, coalesce(returns#94, 0.00) AS returns#97, CheckOverflow((promote_precision(cast(profit#81 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#95, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#98, web channel AS channel#99, wp_web_page_sk#71 AS id#100] -Input [6]: [wp_web_page_sk#71, sales#80, profit#81, wp_web_page_sk#86, returns#94, profit_loss#95] - -(86) Union - -(87) Expand [codegen id : 24] -Input [5]: [sales#17, returns#34, profit#35, channel#36, id#37] -Arguments: [List(sales#17, returns#34, profit#35, channel#36, id#37, 0), List(sales#17, returns#34, profit#35, channel#36, null, 1), List(sales#17, returns#34, profit#35, null, null, 3)], [sales#17, returns#34, profit#35, channel#101, id#102, spark_grouping_id#103] - -(88) HashAggregate [codegen id : 24] -Input [6]: [sales#17, returns#34, profit#35, channel#101, id#102, spark_grouping_id#103] -Keys [3]: [channel#101, id#102, spark_grouping_id#103] -Functions [3]: [partial_sum(sales#17), partial_sum(returns#34), partial_sum(profit#35)] -Aggregate Attributes [6]: [sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Results [9]: [channel#101, id#102, spark_grouping_id#103, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115] - -(89) Exchange -Input [9]: [channel#101, id#102, spark_grouping_id#103, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Arguments: hashpartitioning(channel#101, id#102, spark_grouping_id#103, 5), true, [id=#116] - -(90) HashAggregate [codegen id : 25] -Input [9]: [channel#101, id#102, spark_grouping_id#103, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Keys [3]: [channel#101, id#102, spark_grouping_id#103] -Functions [3]: [sum(sales#17), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#17)#117, sum(returns#34)#118, sum(profit#35)#119] -Results [5]: [channel#101, id#102, sum(sales#17)#117 AS sales#120, sum(returns#34)#118 AS returns#121, sum(profit#35)#119 AS profit#122] - -(91) TakeOrderedAndProject -Input [5]: [channel#101, id#102, sales#120, returns#121, profit#122] -Arguments: 100, [channel#101 ASC NULLS FIRST, id#102 ASC NULLS FIRST], [channel#101, id#102, sales#120, returns#121, profit#122] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q77/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q77/simplified.txt deleted file mode 100644 index bfbeff02b..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q77/simplified.txt +++ /dev/null @@ -1,139 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (25) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (24) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (8) - Project [sales,returns,profit,profit_loss,s_store_sk] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - InputAdapter - Exchange [s_store_sk] #2 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - InputAdapter - Exchange [s_store_sk] #6 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_returned_date_sk,sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_store_sk,sr_return_amt,sr_net_loss] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [s_store_sk] #4 - WholeStageCodegen (15) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - InputAdapter - BroadcastNestedLoopJoin - BroadcastExchange #7 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - InputAdapter - Exchange [cs_call_center_sk] #8 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (14) - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - InputAdapter - Exchange #9 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_return_amount,cr_net_loss] - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (23) - Project [sales,returns,profit,profit_loss,wp_web_page_sk] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - InputAdapter - Exchange [wp_web_page_sk] #10 - WholeStageCodegen (18) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (17) - Filter [wp_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (22) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - InputAdapter - Exchange [wp_web_page_sk] #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returned_date_sk,wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_returned_date_sk,wr_web_page_sk,wr_return_amt,wr_net_loss] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [wp_web_page_sk] #11 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q78/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q78/explain.txt deleted file mode 100644 index 1bcf039dd..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q78/explain.txt +++ /dev/null @@ -1,341 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (60) -+- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * Filter (9) - : : : +- * BroadcastHashJoin LeftOuter BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.store_returns (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.date_dim (11) - : +- BroadcastExchange (37) - : +- * Filter (36) - : +- * HashAggregate (35) - : +- Exchange (34) - : +- * HashAggregate (33) - : +- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * Filter (28) - : : +- * BroadcastHashJoin LeftOuter BuildRight (27) - : : :- * Filter (22) - : : : +- * ColumnarToRow (21) - : : : +- Scan parquet default.web_sales (20) - : : +- BroadcastExchange (26) - : : +- * Filter (25) - : : +- * ColumnarToRow (24) - : : +- Scan parquet default.web_returns (23) - : +- ReusedExchange (30) - +- BroadcastExchange (57) - +- * Filter (56) - +- * HashAggregate (55) - +- Exchange (54) - +- * HashAggregate (53) - +- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (49) - : +- * Filter (48) - : +- * BroadcastHashJoin LeftOuter BuildRight (47) - : :- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.catalog_sales (40) - : +- BroadcastExchange (46) - : +- * Filter (45) - : +- * ColumnarToRow (44) - : +- Scan parquet default.catalog_returns (43) - +- ReusedExchange (50) - - -(1) Scan parquet default.store_sales -Output [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#4, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#4, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7] - -(3) Filter [codegen id : 3] -Input [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#4, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7] -Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) - -(4) Scan parquet default.store_returns -Output [2]: [sr_item_sk#8, sr_ticket_number#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [sr_item_sk#8, sr_ticket_number#9] - -(6) Filter [codegen id : 1] -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) - -(7) BroadcastExchange -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#10] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#2 as bigint)] -Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] -Join condition: None - -(9) Filter [codegen id : 3] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#4, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7, sr_item_sk#8, sr_ticket_number#9] -Condition : isnull(sr_ticket_number#9) - -(10) Project [codegen id : 3] -Output [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#4, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7, sr_item_sk#8, sr_ticket_number#9] - -(11) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#11, d_year#12] - -(13) Filter [codegen id : 2] -Input [2]: [d_date_sk#11, d_year#12] -Condition : ((isnotnull(d_year#12) AND (d_year#12 = 2000)) AND isnotnull(d_date_sk#11)) - -(14) BroadcastExchange -Input [2]: [d_date_sk#11, d_year#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#11] -Join condition: None - -(16) Project [codegen id : 3] -Output [6]: [ss_item_sk#2, ss_customer_sk#3, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7, d_year#12] -Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7, d_date_sk#11, d_year#12] - -(17) HashAggregate [codegen id : 3] -Input [6]: [ss_item_sk#2, ss_customer_sk#3, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7, d_year#12] -Keys [3]: [d_year#12, ss_item_sk#2, ss_customer_sk#3] -Functions [3]: [partial_sum(cast(ss_quantity#5 as bigint)), partial_sum(UnscaledValue(ss_wholesale_cost#6)), partial_sum(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [3]: [sum#14, sum#15, sum#16] -Results [6]: [d_year#12, ss_item_sk#2, ss_customer_sk#3, sum#17, sum#18, sum#19] - -(18) Exchange -Input [6]: [d_year#12, ss_item_sk#2, ss_customer_sk#3, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#12, ss_item_sk#2, ss_customer_sk#3, 5), true, [id=#20] - -(19) HashAggregate [codegen id : 12] -Input [6]: [d_year#12, ss_item_sk#2, ss_customer_sk#3, sum#17, sum#18, sum#19] -Keys [3]: [d_year#12, ss_item_sk#2, ss_customer_sk#3] -Functions [3]: [sum(cast(ss_quantity#5 as bigint)), sum(UnscaledValue(ss_wholesale_cost#6)), sum(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [3]: [sum(cast(ss_quantity#5 as bigint))#21, sum(UnscaledValue(ss_wholesale_cost#6))#22, sum(UnscaledValue(ss_sales_price#7))#23] -Results [6]: [d_year#12 AS ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3, sum(cast(ss_quantity#5 as bigint))#21 AS ss_qty#25, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#6))#22,17,2) AS ss_wc#26, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#23,17,2) AS ss_sp#27] - -(20) Scan parquet default.web_sales -Output [7]: [ws_sold_date_sk#28, ws_item_sk#29, ws_bill_customer_sk#30, ws_order_number#31, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(21) ColumnarToRow [codegen id : 6] -Input [7]: [ws_sold_date_sk#28, ws_item_sk#29, ws_bill_customer_sk#30, ws_order_number#31, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34] - -(22) Filter [codegen id : 6] -Input [7]: [ws_sold_date_sk#28, ws_item_sk#29, ws_bill_customer_sk#30, ws_order_number#31, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34] -Condition : ((isnotnull(ws_sold_date_sk#28) AND isnotnull(ws_item_sk#29)) AND isnotnull(ws_bill_customer_sk#30)) - -(23) Scan parquet default.web_returns -Output [2]: [wr_item_sk#35, wr_order_number#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [wr_item_sk#35, wr_order_number#36] - -(25) Filter [codegen id : 4] -Input [2]: [wr_item_sk#35, wr_order_number#36] -Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) - -(26) BroadcastExchange -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false], input[0, bigint, false]),false), [id=#37] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [cast(ws_order_number#31 as bigint), cast(ws_item_sk#29 as bigint)] -Right keys [2]: [wr_order_number#36, wr_item_sk#35] -Join condition: None - -(28) Filter [codegen id : 6] -Input [9]: [ws_sold_date_sk#28, ws_item_sk#29, ws_bill_customer_sk#30, ws_order_number#31, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34, wr_item_sk#35, wr_order_number#36] -Condition : isnull(wr_order_number#36) - -(29) Project [codegen id : 6] -Output [6]: [ws_sold_date_sk#28, ws_item_sk#29, ws_bill_customer_sk#30, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34] -Input [9]: [ws_sold_date_sk#28, ws_item_sk#29, ws_bill_customer_sk#30, ws_order_number#31, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34, wr_item_sk#35, wr_order_number#36] - -(30) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#11, d_year#12] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#28] -Right keys [1]: [d_date_sk#11] -Join condition: None - -(32) Project [codegen id : 6] -Output [6]: [ws_item_sk#29, ws_bill_customer_sk#30, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34, d_year#12] -Input [8]: [ws_sold_date_sk#28, ws_item_sk#29, ws_bill_customer_sk#30, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34, d_date_sk#11, d_year#12] - -(33) HashAggregate [codegen id : 6] -Input [6]: [ws_item_sk#29, ws_bill_customer_sk#30, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34, d_year#12] -Keys [3]: [d_year#12, ws_item_sk#29, ws_bill_customer_sk#30] -Functions [3]: [partial_sum(cast(ws_quantity#32 as bigint)), partial_sum(UnscaledValue(ws_wholesale_cost#33)), partial_sum(UnscaledValue(ws_sales_price#34))] -Aggregate Attributes [3]: [sum#38, sum#39, sum#40] -Results [6]: [d_year#12, ws_item_sk#29, ws_bill_customer_sk#30, sum#41, sum#42, sum#43] - -(34) Exchange -Input [6]: [d_year#12, ws_item_sk#29, ws_bill_customer_sk#30, sum#41, sum#42, sum#43] -Arguments: hashpartitioning(d_year#12, ws_item_sk#29, ws_bill_customer_sk#30, 5), true, [id=#44] - -(35) HashAggregate [codegen id : 7] -Input [6]: [d_year#12, ws_item_sk#29, ws_bill_customer_sk#30, sum#41, sum#42, sum#43] -Keys [3]: [d_year#12, ws_item_sk#29, ws_bill_customer_sk#30] -Functions [3]: [sum(cast(ws_quantity#32 as bigint)), sum(UnscaledValue(ws_wholesale_cost#33)), sum(UnscaledValue(ws_sales_price#34))] -Aggregate Attributes [3]: [sum(cast(ws_quantity#32 as bigint))#45, sum(UnscaledValue(ws_wholesale_cost#33))#46, sum(UnscaledValue(ws_sales_price#34))#47] -Results [6]: [d_year#12 AS ws_sold_year#48, ws_item_sk#29, ws_bill_customer_sk#30 AS ws_customer_sk#49, sum(cast(ws_quantity#32 as bigint))#45 AS ws_qty#50, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#33))#46,17,2) AS ws_wc#51, MakeDecimal(sum(UnscaledValue(ws_sales_price#34))#47,17,2) AS ws_sp#52] - -(36) Filter [codegen id : 7] -Input [6]: [ws_sold_year#48, ws_item_sk#29, ws_customer_sk#49, ws_qty#50, ws_wc#51, ws_sp#52] -Condition : (coalesce(ws_qty#50, 0) > 0) - -(37) BroadcastExchange -Input [6]: [ws_sold_year#48, ws_item_sk#29, ws_customer_sk#49, ws_qty#50, ws_wc#51, ws_sp#52] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#53] - -(38) BroadcastHashJoin [codegen id : 12] -Left keys [3]: [ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3] -Right keys [3]: [ws_sold_year#48, ws_item_sk#29, ws_customer_sk#49] -Join condition: None - -(39) Project [codegen id : 12] -Output [9]: [ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3, ss_qty#25, ss_wc#26, ss_sp#27, ws_qty#50, ws_wc#51, ws_sp#52] -Input [12]: [ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3, ss_qty#25, ss_wc#26, ss_sp#27, ws_sold_year#48, ws_item_sk#29, ws_customer_sk#49, ws_qty#50, ws_wc#51, ws_sp#52] - -(40) Scan parquet default.catalog_sales -Output [7]: [cs_sold_date_sk#54, cs_bill_customer_sk#55, cs_item_sk#56, cs_order_number#57, cs_quantity#58, cs_wholesale_cost#59, cs_sales_price#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 10] -Input [7]: [cs_sold_date_sk#54, cs_bill_customer_sk#55, cs_item_sk#56, cs_order_number#57, cs_quantity#58, cs_wholesale_cost#59, cs_sales_price#60] - -(42) Filter [codegen id : 10] -Input [7]: [cs_sold_date_sk#54, cs_bill_customer_sk#55, cs_item_sk#56, cs_order_number#57, cs_quantity#58, cs_wholesale_cost#59, cs_sales_price#60] -Condition : ((isnotnull(cs_sold_date_sk#54) AND isnotnull(cs_item_sk#56)) AND isnotnull(cs_bill_customer_sk#55)) - -(43) Scan parquet default.catalog_returns -Output [2]: [cr_item_sk#61, cr_order_number#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 8] -Input [2]: [cr_item_sk#61, cr_order_number#62] - -(45) Filter [codegen id : 8] -Input [2]: [cr_item_sk#61, cr_order_number#62] -Condition : (isnotnull(cr_order_number#62) AND isnotnull(cr_item_sk#61)) - -(46) BroadcastExchange -Input [2]: [cr_item_sk#61, cr_order_number#62] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [id=#63] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#57, cs_item_sk#56] -Right keys [2]: [cr_order_number#62, cr_item_sk#61] -Join condition: None - -(48) Filter [codegen id : 10] -Input [9]: [cs_sold_date_sk#54, cs_bill_customer_sk#55, cs_item_sk#56, cs_order_number#57, cs_quantity#58, cs_wholesale_cost#59, cs_sales_price#60, cr_item_sk#61, cr_order_number#62] -Condition : isnull(cr_order_number#62) - -(49) Project [codegen id : 10] -Output [6]: [cs_sold_date_sk#54, cs_bill_customer_sk#55, cs_item_sk#56, cs_quantity#58, cs_wholesale_cost#59, cs_sales_price#60] -Input [9]: [cs_sold_date_sk#54, cs_bill_customer_sk#55, cs_item_sk#56, cs_order_number#57, cs_quantity#58, cs_wholesale_cost#59, cs_sales_price#60, cr_item_sk#61, cr_order_number#62] - -(50) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#11, d_year#12] - -(51) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#54] -Right keys [1]: [d_date_sk#11] -Join condition: None - -(52) Project [codegen id : 10] -Output [6]: [cs_bill_customer_sk#55, cs_item_sk#56, cs_quantity#58, cs_wholesale_cost#59, cs_sales_price#60, d_year#12] -Input [8]: [cs_sold_date_sk#54, cs_bill_customer_sk#55, cs_item_sk#56, cs_quantity#58, cs_wholesale_cost#59, cs_sales_price#60, d_date_sk#11, d_year#12] - -(53) HashAggregate [codegen id : 10] -Input [6]: [cs_bill_customer_sk#55, cs_item_sk#56, cs_quantity#58, cs_wholesale_cost#59, cs_sales_price#60, d_year#12] -Keys [3]: [d_year#12, cs_item_sk#56, cs_bill_customer_sk#55] -Functions [3]: [partial_sum(cast(cs_quantity#58 as bigint)), partial_sum(UnscaledValue(cs_wholesale_cost#59)), partial_sum(UnscaledValue(cs_sales_price#60))] -Aggregate Attributes [3]: [sum#64, sum#65, sum#66] -Results [6]: [d_year#12, cs_item_sk#56, cs_bill_customer_sk#55, sum#67, sum#68, sum#69] - -(54) Exchange -Input [6]: [d_year#12, cs_item_sk#56, cs_bill_customer_sk#55, sum#67, sum#68, sum#69] -Arguments: hashpartitioning(d_year#12, cs_item_sk#56, cs_bill_customer_sk#55, 5), true, [id=#70] - -(55) HashAggregate [codegen id : 11] -Input [6]: [d_year#12, cs_item_sk#56, cs_bill_customer_sk#55, sum#67, sum#68, sum#69] -Keys [3]: [d_year#12, cs_item_sk#56, cs_bill_customer_sk#55] -Functions [3]: [sum(cast(cs_quantity#58 as bigint)), sum(UnscaledValue(cs_wholesale_cost#59)), sum(UnscaledValue(cs_sales_price#60))] -Aggregate Attributes [3]: [sum(cast(cs_quantity#58 as bigint))#71, sum(UnscaledValue(cs_wholesale_cost#59))#72, sum(UnscaledValue(cs_sales_price#60))#73] -Results [6]: [d_year#12 AS cs_sold_year#74, cs_item_sk#56, cs_bill_customer_sk#55 AS cs_customer_sk#75, sum(cast(cs_quantity#58 as bigint))#71 AS cs_qty#76, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#59))#72,17,2) AS cs_wc#77, MakeDecimal(sum(UnscaledValue(cs_sales_price#60))#73,17,2) AS cs_sp#78] - -(56) Filter [codegen id : 11] -Input [6]: [cs_sold_year#74, cs_item_sk#56, cs_customer_sk#75, cs_qty#76, cs_wc#77, cs_sp#78] -Condition : (coalesce(cs_qty#76, 0) > 0) - -(57) BroadcastExchange -Input [6]: [cs_sold_year#74, cs_item_sk#56, cs_customer_sk#75, cs_qty#76, cs_wc#77, cs_sp#78] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#79] - -(58) BroadcastHashJoin [codegen id : 12] -Left keys [3]: [ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3] -Right keys [3]: [cs_sold_year#74, cs_item_sk#56, cs_customer_sk#75] -Join condition: None - -(59) Project [codegen id : 12] -Output [12]: [round((cast(ss_qty#25 as double) / cast(coalesce((ws_qty#50 + cs_qty#76), 1) as double)), 2) AS ratio#80, ss_qty#25 AS store_qty#81, ss_wc#26 AS store_wholesale_cost#82, ss_sp#27 AS store_sales_price#83, (coalesce(ws_qty#50, 0) + coalesce(cs_qty#76, 0)) AS other_chan_qty#84, CheckOverflow((promote_precision(cast(coalesce(ws_wc#51, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#77, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#85, CheckOverflow((promote_precision(cast(coalesce(ws_sp#52, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#78, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#86, ss_qty#25, ss_wc#26, ss_sp#27, ws_qty#50, cs_qty#76] -Input [15]: [ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3, ss_qty#25, ss_wc#26, ss_sp#27, ws_qty#50, ws_wc#51, ws_sp#52, cs_sold_year#74, cs_item_sk#56, cs_customer_sk#75, cs_qty#76, cs_wc#77, cs_sp#78] - -(60) TakeOrderedAndProject -Input [12]: [ratio#80, store_qty#81, store_wholesale_cost#82, store_sales_price#83, other_chan_qty#84, other_chan_wholesale_cost#85, other_chan_sales_price#86, ss_qty#25, ss_wc#26, ss_sp#27, ws_qty#50, cs_qty#76] -Arguments: 100, [ratio#80 ASC NULLS FIRST, ss_qty#25 DESC NULLS LAST, ss_wc#26 DESC NULLS LAST, ss_sp#27 DESC NULLS LAST, other_chan_qty#84 ASC NULLS FIRST, other_chan_wholesale_cost#85 ASC NULLS FIRST, other_chan_sales_price#86 ASC NULLS FIRST, round((cast(ss_qty#25 as double) / cast(coalesce((ws_qty#50 + cs_qty#76), 1) as double)), 2) ASC NULLS FIRST], [ratio#80, store_qty#81, store_wholesale_cost#82, store_sales_price#83, other_chan_qty#84, other_chan_wholesale_cost#85, other_chan_sales_price#86] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q78/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q78/simplified.txt deleted file mode 100644 index 977070283..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q78/simplified.txt +++ /dev/null @@ -1,88 +0,0 @@ -TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (12) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - BroadcastHashJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] - Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - BroadcastHashJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(cast(ss_quantity as bigint)),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (3) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] - Filter [sr_ticket_number] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - Filter [ws_qty] - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(cast(ws_quantity as bigint)),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (6) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] - Filter [wr_order_number] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - Filter [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [wr_order_number,wr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - Filter [cs_qty] - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cast(cs_quantity as bigint)),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (10) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] - Filter [cr_order_number] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - Filter [cs_sold_date_sk,cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Filter [cr_order_number,cr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q79/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q79/explain.txt deleted file mode 100644 index 63cde7268..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q79/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.store (11) - : +- BroadcastExchange (22) - : +- * Project (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.household_demographics (18) - +- BroadcastExchange (31) - +- * Filter (30) - +- * ColumnarToRow (29) - +- Scan parquet default.customer (28) - - -(1) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [8]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8] - -(3) Filter [codegen id : 4] -Input [8]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8] -Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#5)) AND isnotnull(ss_hdemo_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_dow#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#9, d_year#10, d_dow#11] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#9, d_year#10, d_dow#11] -Condition : (((isnotnull(d_dow#11) AND (d_dow#11 = 1)) AND d_year#10 IN (1999,2000,2001)) AND isnotnull(d_date_sk#9)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#9] -Input [3]: [d_date_sk#9, d_year#10, d_dow#11] - -(8) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#9] -Join condition: None - -(10) Project [codegen id : 4] -Output [7]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8] -Input [9]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, d_date_sk#9] - -(11) Scan parquet default.store -Output [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] - -(13) Filter [codegen id : 2] -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Condition : (((isnotnull(s_number_employees#14) AND (s_number_employees#14 >= 200)) AND (s_number_employees#14 <= 295)) AND isnotnull(s_store_sk#13)) - -(14) Project [codegen id : 2] -Output [2]: [s_store_sk#13, s_city#15] -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] - -(15) BroadcastExchange -Input [2]: [s_store_sk#13, s_city#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#13] -Join condition: None - -(17) Project [codegen id : 4] -Output [7]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, s_city#15] -Input [9]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, s_store_sk#13, s_city#15] - -(18) Scan parquet default.household_demographics -Output [3]: [hd_demo_sk#17, hd_dep_count#18, hd_vehicle_count#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [3]: [hd_demo_sk#17, hd_dep_count#18, hd_vehicle_count#19] - -(20) Filter [codegen id : 3] -Input [3]: [hd_demo_sk#17, hd_dep_count#18, hd_vehicle_count#19] -Condition : (((hd_dep_count#18 = 6) OR (hd_vehicle_count#19 > 2)) AND isnotnull(hd_demo_sk#17)) - -(21) Project [codegen id : 3] -Output [1]: [hd_demo_sk#17] -Input [3]: [hd_demo_sk#17, hd_dep_count#18, hd_vehicle_count#19] - -(22) BroadcastExchange -Input [1]: [hd_demo_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#17] -Join condition: None - -(24) Project [codegen id : 4] -Output [6]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, s_city#15] -Input [8]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, s_city#15, hd_demo_sk#17] - -(25) HashAggregate [codegen id : 4] -Input [6]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, s_city#15] -Keys [4]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, s_city#15] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#7)), partial_sum(UnscaledValue(ss_net_profit#8))] -Aggregate Attributes [2]: [sum#21, sum#22] -Results [6]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, s_city#15, sum#23, sum#24] - -(26) Exchange -Input [6]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, s_city#15, sum#23, sum#24] -Arguments: hashpartitioning(ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, s_city#15, 5), true, [id=#25] - -(27) HashAggregate [codegen id : 6] -Input [6]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, s_city#15, sum#23, sum#24] -Keys [4]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, s_city#15] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#7)), sum(UnscaledValue(ss_net_profit#8))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#7))#26, sum(UnscaledValue(ss_net_profit#8))#27] -Results [5]: [ss_ticket_number#6, ss_customer_sk#2, s_city#15, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#7))#26,17,2) AS amt#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#8))#27,17,2) AS profit#29] - -(28) Scan parquet default.customer -Output [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] - -(30) Filter [codegen id : 5] -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Condition : isnotnull(c_customer_sk#30) - -(31) BroadcastExchange -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] - -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#30] -Join condition: None - -(33) Project [codegen id : 6] -Output [7]: [c_last_name#32, c_first_name#31, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#34, ss_ticket_number#6, amt#28, profit#29, s_city#15] -Input [8]: [ss_ticket_number#6, ss_customer_sk#2, s_city#15, amt#28, profit#29, c_customer_sk#30, c_first_name#31, c_last_name#32] - -(34) TakeOrderedAndProject -Input [7]: [c_last_name#32, c_first_name#31, substr(s_city, 1, 30)#34, ss_ticket_number#6, amt#28, profit#29, s_city#15] -Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#29 ASC NULLS FIRST], [c_last_name#32, c_first_name#31, substr(s_city, 1, 30)#34, ss_ticket_number#6, amt#28, profit#29] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q79/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q79/simplified.txt deleted file mode 100644 index 5b6177488..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q79/simplified.txt +++ /dev/null @@ -1,50 +0,0 @@ -TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (6) - Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] - InputAdapter - Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_dow,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [s_store_sk,s_city] - Filter [s_number_employees,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_number_employees,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [hd_demo_sk] - Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q8/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q8/explain.txt deleted file mode 100644 index e9f9a92a0..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q8/explain.txt +++ /dev/null @@ -1,272 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.date_dim (4) - : +- BroadcastExchange (14) - : +- * Filter (13) - : +- * ColumnarToRow (12) - : +- Scan parquet default.store (11) - +- BroadcastExchange (41) - +- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * Project (37) - +- * BroadcastHashJoin LeftSemi BuildRight (36) - :- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet default.customer_address (17) - +- BroadcastExchange (35) - +- * Project (34) - +- * Filter (33) - +- * HashAggregate (32) - +- Exchange (31) - +- * HashAggregate (30) - +- * Project (29) - +- * BroadcastHashJoin Inner BuildRight (28) - :- * Filter (22) - : +- * ColumnarToRow (21) - : +- Scan parquet default.customer_address (20) - +- BroadcastExchange (27) - +- * Project (26) - +- * Filter (25) - +- * ColumnarToRow (24) - +- Scan parquet default.customer (23) - - -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] - -(3) Filter [codegen id : 8] -Input [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#4, d_year#5, d_qoy#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] -Condition : ((((isnotnull(d_qoy#6) AND isnotnull(d_year#5)) AND (d_qoy#6 = 2)) AND (d_year#5 = 1998)) AND isnotnull(d_date_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] - -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(10) Project [codegen id : 8] -Output [2]: [ss_store_sk#2, ss_net_profit#3] -Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3, d_date_sk#4] - -(11) Scan parquet default.store -Output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_zip)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] - -(13) Filter [codegen id : 2] -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_zip#10)) - -(14) BroadcastExchange -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] - -(15) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#8] -Join condition: None - -(16) Project [codegen id : 8] -Output [3]: [ss_net_profit#3, s_store_name#9, s_zip#10] -Input [5]: [ss_store_sk#2, ss_net_profit#3, s_store_sk#8, s_store_name#9, s_zip#10] - -(17) Scan parquet default.customer_address -Output [1]: [ca_zip#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [1]: [ca_zip#12] - -(19) Filter [codegen id : 6] -Input [1]: [ca_zip#12] -Condition : (substr(ca_zip#12, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#12, 1, 5))) - -(20) Scan parquet default.customer_address -Output [2]: [ca_address_sk#13, ca_zip#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#13, ca_zip#12] - -(22) Filter [codegen id : 4] -Input [2]: [ca_address_sk#13, ca_zip#12] -Condition : isnotnull(ca_address_sk#13) - -(23) Scan parquet default.customer -Output [2]: [c_current_addr_sk#14, c_preferred_cust_flag#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 3] -Input [2]: [c_current_addr_sk#14, c_preferred_cust_flag#15] - -(25) Filter [codegen id : 3] -Input [2]: [c_current_addr_sk#14, c_preferred_cust_flag#15] -Condition : ((isnotnull(c_preferred_cust_flag#15) AND (c_preferred_cust_flag#15 = Y)) AND isnotnull(c_current_addr_sk#14)) - -(26) Project [codegen id : 3] -Output [1]: [c_current_addr_sk#14] -Input [2]: [c_current_addr_sk#14, c_preferred_cust_flag#15] - -(27) BroadcastExchange -Input [1]: [c_current_addr_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(28) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ca_address_sk#13] -Right keys [1]: [c_current_addr_sk#14] -Join condition: None - -(29) Project [codegen id : 4] -Output [1]: [ca_zip#12] -Input [3]: [ca_address_sk#13, ca_zip#12, c_current_addr_sk#14] - -(30) HashAggregate [codegen id : 4] -Input [1]: [ca_zip#12] -Keys [1]: [ca_zip#12] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#17] -Results [2]: [ca_zip#12, count#18] - -(31) Exchange -Input [2]: [ca_zip#12, count#18] -Arguments: hashpartitioning(ca_zip#12, 5), true, [id=#19] - -(32) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#12, count#18] -Keys [1]: [ca_zip#12] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [2]: [substr(ca_zip#12, 1, 5) AS ca_zip#21, count(1)#20 AS count(1)#22] - -(33) Filter [codegen id : 5] -Input [2]: [ca_zip#21, count(1)#22] -Condition : (count(1)#22 > 10) - -(34) Project [codegen id : 5] -Output [1]: [ca_zip#21] -Input [2]: [ca_zip#21, count(1)#22] - -(35) BroadcastExchange -Input [1]: [ca_zip#21] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#23] - -(36) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [coalesce(substr(ca_zip#12, 1, 5), ), isnull(substr(ca_zip#12, 1, 5))] -Right keys [2]: [coalesce(ca_zip#21, ), isnull(ca_zip#21)] -Join condition: None - -(37) Project [codegen id : 6] -Output [1]: [substr(ca_zip#12, 1, 5) AS ca_zip#24] -Input [1]: [ca_zip#12] - -(38) HashAggregate [codegen id : 6] -Input [1]: [ca_zip#24] -Keys [1]: [ca_zip#24] -Functions: [] -Aggregate Attributes: [] -Results [1]: [ca_zip#24] - -(39) Exchange -Input [1]: [ca_zip#24] -Arguments: hashpartitioning(ca_zip#24, 5), true, [id=#25] - -(40) HashAggregate [codegen id : 7] -Input [1]: [ca_zip#24] -Keys [1]: [ca_zip#24] -Functions: [] -Aggregate Attributes: [] -Results [1]: [ca_zip#24] - -(41) BroadcastExchange -Input [1]: [ca_zip#24] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#26] - -(42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [substr(s_zip#10, 1, 2)] -Right keys [1]: [substr(ca_zip#24, 1, 2)] -Join condition: None - -(43) Project [codegen id : 8] -Output [2]: [ss_net_profit#3, s_store_name#9] -Input [4]: [ss_net_profit#3, s_store_name#9, s_zip#10, ca_zip#24] - -(44) HashAggregate [codegen id : 8] -Input [2]: [ss_net_profit#3, s_store_name#9] -Keys [1]: [s_store_name#9] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [s_store_name#9, sum#28] - -(45) Exchange -Input [2]: [s_store_name#9, sum#28] -Arguments: hashpartitioning(s_store_name#9, 5), true, [id=#29] - -(46) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#9, sum#28] -Keys [1]: [s_store_name#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#3))#30] -Results [2]: [s_store_name#9, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#30,17,2) AS sum(ss_net_profit)#31] - -(47) TakeOrderedAndProject -Input [2]: [s_store_name#9, sum(ss_net_profit)#31] -Arguments: 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#31] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q8/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q8/simplified.txt deleted file mode 100644 index cc62907dc..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q8/simplified.txt +++ /dev/null @@ -1,70 +0,0 @@ -TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (9) - HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - InputAdapter - Exchange [s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_store_name] - BroadcastHashJoin [s_zip,ca_zip] - Project [ss_net_profit,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [s_store_sk,s_zip] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [ca_zip] - InputAdapter - Exchange [ca_zip] #5 - WholeStageCodegen (6) - HashAggregate [ca_zip] - Project [ca_zip] - BroadcastHashJoin [ca_zip,ca_zip] - Filter [ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_zip] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [ca_zip] - Filter [count(1)] - HashAggregate [ca_zip,count] [count(1),ca_zip,count(1),count] - InputAdapter - Exchange [ca_zip] #7 - WholeStageCodegen (4) - HashAggregate [ca_zip] [count,count] - Project [ca_zip] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_zip] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [c_current_addr_sk] - Filter [c_preferred_cust_flag,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q80/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q80/explain.txt deleted file mode 100644 index 36b045bfd..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q80/explain.txt +++ /dev/null @@ -1,553 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (99) -+- * HashAggregate (98) - +- Exchange (97) - +- * HashAggregate (96) - +- * Expand (95) - +- Union (94) - :- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * Project (16) - : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin LeftOuter BuildRight (8) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet default.store_sales (1) - : : : : : +- BroadcastExchange (7) - : : : : : +- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_returns (4) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * Filter (12) - : : : : +- * ColumnarToRow (11) - : : : : +- Scan parquet default.date_dim (10) - : : : +- BroadcastExchange (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.store (17) - : : +- BroadcastExchange (27) - : : +- * Project (26) - : : +- * Filter (25) - : : +- * ColumnarToRow (24) - : : +- Scan parquet default.item (23) - : +- BroadcastExchange (34) - : +- * Project (33) - : +- * Filter (32) - : +- * ColumnarToRow (31) - : +- Scan parquet default.promotion (30) - :- * HashAggregate (66) - : +- Exchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : :- * Project (51) - : : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : : :- * Project (48) - : : : : : +- * BroadcastHashJoin LeftOuter BuildRight (47) - : : : : : :- * Filter (42) - : : : : : : +- * ColumnarToRow (41) - : : : : : : +- Scan parquet default.catalog_sales (40) - : : : : : +- BroadcastExchange (46) - : : : : : +- * Filter (45) - : : : : : +- * ColumnarToRow (44) - : : : : : +- Scan parquet default.catalog_returns (43) - : : : : +- ReusedExchange (49) - : : : +- BroadcastExchange (55) - : : : +- * Filter (54) - : : : +- * ColumnarToRow (53) - : : : +- Scan parquet default.catalog_page (52) - : : +- ReusedExchange (58) - : +- ReusedExchange (61) - +- * HashAggregate (93) - +- Exchange (92) - +- * HashAggregate (91) - +- * Project (90) - +- * BroadcastHashJoin Inner BuildRight (89) - :- * Project (87) - : +- * BroadcastHashJoin Inner BuildRight (86) - : :- * Project (84) - : : +- * BroadcastHashJoin Inner BuildRight (83) - : : :- * Project (78) - : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : :- * Project (75) - : : : : +- * BroadcastHashJoin LeftOuter BuildRight (74) - : : : : :- * Filter (69) - : : : : : +- * ColumnarToRow (68) - : : : : : +- Scan parquet default.web_sales (67) - : : : : +- BroadcastExchange (73) - : : : : +- * Filter (72) - : : : : +- * ColumnarToRow (71) - : : : : +- Scan parquet default.web_returns (70) - : : : +- ReusedExchange (76) - : : +- BroadcastExchange (82) - : : +- * Filter (81) - : : +- * ColumnarToRow (80) - : : +- Scan parquet default.web_site (79) - : +- ReusedExchange (85) - +- ReusedExchange (88) - - -(1) Scan parquet default.store_sales -Output [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_net_profit#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_net_profit#7] - -(3) Filter [codegen id : 6] -Input [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_net_profit#7] -Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_promo_sk#4)) - -(4) Scan parquet default.store_returns -Output [4]: [sr_item_sk#8, sr_ticket_number#9, sr_return_amt#10, sr_net_loss#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [sr_item_sk#8, sr_ticket_number#9, sr_return_amt#10, sr_net_loss#11] - -(6) Filter [codegen id : 1] -Input [4]: [sr_item_sk#8, sr_ticket_number#9, sr_return_amt#10, sr_net_loss#11] -Condition : (isnotnull(sr_item_sk#8) AND isnotnull(sr_ticket_number#9)) - -(7) BroadcastExchange -Input [4]: [sr_item_sk#8, sr_ticket_number#9, sr_return_amt#10, sr_net_loss#11] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[1, bigint, false]),false), [id=#12] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [2]: [sr_item_sk#8, sr_ticket_number#9] -Join condition: None - -(9) Project [codegen id : 6] -Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#10, sr_net_loss#11] -Input [11]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_net_profit#7, sr_item_sk#8, sr_ticket_number#9, sr_return_amt#10, sr_net_loss#11] - -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#13, d_date#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#13, d_date#14] - -(12) Filter [codegen id : 2] -Input [2]: [d_date_sk#13, d_date#14] -Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 11192)) AND (d_date#14 <= 11222)) AND isnotnull(d_date_sk#13)) - -(13) Project [codegen id : 2] -Output [1]: [d_date_sk#13] -Input [2]: [d_date_sk#13, d_date#14] - -(14) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(16) Project [codegen id : 6] -Output [7]: [ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#10, sr_net_loss#11] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#10, sr_net_loss#11, d_date_sk#13] - -(17) Scan parquet default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#16, s_store_id#17] - -(19) Filter [codegen id : 3] -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) BroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#16] -Join condition: None - -(22) Project [codegen id : 6] -Output [7]: [ss_item_sk#2, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#10, sr_net_loss#11, s_store_id#17] -Input [9]: [ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#10, sr_net_loss#11, s_store_sk#16, s_store_id#17] - -(23) Scan parquet default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#19, i_current_price#20] - -(25) Filter [codegen id : 4] -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) Project [codegen id : 4] -Output [1]: [i_item_sk#19] -Input [2]: [i_item_sk#19, i_current_price#20] - -(27) BroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#19] -Join condition: None - -(29) Project [codegen id : 6] -Output [6]: [ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#10, sr_net_loss#11, s_store_id#17] -Input [8]: [ss_item_sk#2, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#10, sr_net_loss#11, s_store_id#17, i_item_sk#19] - -(30) Scan parquet default.promotion -Output [2]: [p_promo_sk#22, p_channel_tv#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 5] -Input [2]: [p_promo_sk#22, p_channel_tv#23] - -(32) Filter [codegen id : 5] -Input [2]: [p_promo_sk#22, p_channel_tv#23] -Condition : ((isnotnull(p_channel_tv#23) AND (p_channel_tv#23 = N)) AND isnotnull(p_promo_sk#22)) - -(33) Project [codegen id : 5] -Output [1]: [p_promo_sk#22] -Input [2]: [p_promo_sk#22, p_channel_tv#23] - -(34) BroadcastExchange -Input [1]: [p_promo_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] - -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#22] -Join condition: None - -(36) Project [codegen id : 6] -Output [5]: [ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#10, sr_net_loss#11, s_store_id#17] -Input [7]: [ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#10, sr_net_loss#11, s_store_id#17, p_promo_sk#22] - -(37) HashAggregate [codegen id : 6] -Input [5]: [ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#10, sr_net_loss#11, s_store_id#17] -Keys [1]: [s_store_id#17] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(coalesce(cast(sr_return_amt#10 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ss_net_profit#7 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#11 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#25, sum#26, isEmpty#27, sum#28, isEmpty#29] -Results [6]: [s_store_id#17, sum#30, sum#31, isEmpty#32, sum#33, isEmpty#34] - -(38) Exchange -Input [6]: [s_store_id#17, sum#30, sum#31, isEmpty#32, sum#33, isEmpty#34] -Arguments: hashpartitioning(s_store_id#17, 5), true, [id=#35] - -(39) HashAggregate [codegen id : 7] -Input [6]: [s_store_id#17, sum#30, sum#31, isEmpty#32, sum#33, isEmpty#34] -Keys [1]: [s_store_id#17] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(coalesce(cast(sr_return_amt#10 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ss_net_profit#7 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#11 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#36, sum(coalesce(cast(sr_return_amt#10 as decimal(12,2)), 0.00))#37, sum(CheckOverflow((promote_precision(cast(ss_net_profit#7 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#11 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#38] -Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#36,17,2) AS sales#39, sum(coalesce(cast(sr_return_amt#10 as decimal(12,2)), 0.00))#37 AS returns#40, sum(CheckOverflow((promote_precision(cast(ss_net_profit#7 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#11 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#38 AS profit#41, store channel AS channel#42, concat(store, s_store_id#17) AS id#43] - -(40) Scan parquet default.catalog_sales -Output [7]: [cs_sold_date_sk#44, cs_catalog_page_sk#45, cs_item_sk#46, cs_promo_sk#47, cs_order_number#48, cs_ext_sales_price#49, cs_net_profit#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 13] -Input [7]: [cs_sold_date_sk#44, cs_catalog_page_sk#45, cs_item_sk#46, cs_promo_sk#47, cs_order_number#48, cs_ext_sales_price#49, cs_net_profit#50] - -(42) Filter [codegen id : 13] -Input [7]: [cs_sold_date_sk#44, cs_catalog_page_sk#45, cs_item_sk#46, cs_promo_sk#47, cs_order_number#48, cs_ext_sales_price#49, cs_net_profit#50] -Condition : (((isnotnull(cs_sold_date_sk#44) AND isnotnull(cs_catalog_page_sk#45)) AND isnotnull(cs_item_sk#46)) AND isnotnull(cs_promo_sk#47)) - -(43) Scan parquet default.catalog_returns -Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_amount#53, cr_net_loss#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 8] -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_amount#53, cr_net_loss#54] - -(45) Filter [codegen id : 8] -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_amount#53, cr_net_loss#54] -Condition : (isnotnull(cr_item_sk#51) AND isnotnull(cr_order_number#52)) - -(46) BroadcastExchange -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_amount#53, cr_net_loss#54] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [id=#55] - -(47) BroadcastHashJoin [codegen id : 13] -Left keys [2]: [cs_item_sk#46, cs_order_number#48] -Right keys [2]: [cr_item_sk#51, cr_order_number#52] -Join condition: None - -(48) Project [codegen id : 13] -Output [8]: [cs_sold_date_sk#44, cs_catalog_page_sk#45, cs_item_sk#46, cs_promo_sk#47, cs_ext_sales_price#49, cs_net_profit#50, cr_return_amount#53, cr_net_loss#54] -Input [11]: [cs_sold_date_sk#44, cs_catalog_page_sk#45, cs_item_sk#46, cs_promo_sk#47, cs_order_number#48, cs_ext_sales_price#49, cs_net_profit#50, cr_item_sk#51, cr_order_number#52, cr_return_amount#53, cr_net_loss#54] - -(49) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#13] - -(50) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_sold_date_sk#44] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(51) Project [codegen id : 13] -Output [7]: [cs_catalog_page_sk#45, cs_item_sk#46, cs_promo_sk#47, cs_ext_sales_price#49, cs_net_profit#50, cr_return_amount#53, cr_net_loss#54] -Input [9]: [cs_sold_date_sk#44, cs_catalog_page_sk#45, cs_item_sk#46, cs_promo_sk#47, cs_ext_sales_price#49, cs_net_profit#50, cr_return_amount#53, cr_net_loss#54, d_date_sk#13] - -(52) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#56, cp_catalog_page_id#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(53) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#56, cp_catalog_page_id#57] - -(54) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#56, cp_catalog_page_id#57] -Condition : isnotnull(cp_catalog_page_sk#56) - -(55) BroadcastExchange -Input [2]: [cp_catalog_page_sk#56, cp_catalog_page_id#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] - -(56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_catalog_page_sk#45] -Right keys [1]: [cp_catalog_page_sk#56] -Join condition: None - -(57) Project [codegen id : 13] -Output [7]: [cs_item_sk#46, cs_promo_sk#47, cs_ext_sales_price#49, cs_net_profit#50, cr_return_amount#53, cr_net_loss#54, cp_catalog_page_id#57] -Input [9]: [cs_catalog_page_sk#45, cs_item_sk#46, cs_promo_sk#47, cs_ext_sales_price#49, cs_net_profit#50, cr_return_amount#53, cr_net_loss#54, cp_catalog_page_sk#56, cp_catalog_page_id#57] - -(58) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#19] - -(59) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_item_sk#46] -Right keys [1]: [i_item_sk#19] -Join condition: None - -(60) Project [codegen id : 13] -Output [6]: [cs_promo_sk#47, cs_ext_sales_price#49, cs_net_profit#50, cr_return_amount#53, cr_net_loss#54, cp_catalog_page_id#57] -Input [8]: [cs_item_sk#46, cs_promo_sk#47, cs_ext_sales_price#49, cs_net_profit#50, cr_return_amount#53, cr_net_loss#54, cp_catalog_page_id#57, i_item_sk#19] - -(61) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#22] - -(62) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_promo_sk#47] -Right keys [1]: [p_promo_sk#22] -Join condition: None - -(63) Project [codegen id : 13] -Output [5]: [cs_ext_sales_price#49, cs_net_profit#50, cr_return_amount#53, cr_net_loss#54, cp_catalog_page_id#57] -Input [7]: [cs_promo_sk#47, cs_ext_sales_price#49, cs_net_profit#50, cr_return_amount#53, cr_net_loss#54, cp_catalog_page_id#57, p_promo_sk#22] - -(64) HashAggregate [codegen id : 13] -Input [5]: [cs_ext_sales_price#49, cs_net_profit#50, cr_return_amount#53, cr_net_loss#54, cp_catalog_page_id#57] -Keys [1]: [cp_catalog_page_id#57] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#49)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#50 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#54 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#59, sum#60, isEmpty#61, sum#62, isEmpty#63] -Results [6]: [cp_catalog_page_id#57, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] - -(65) Exchange -Input [6]: [cp_catalog_page_id#57, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Arguments: hashpartitioning(cp_catalog_page_id#57, 5), true, [id=#69] - -(66) HashAggregate [codegen id : 14] -Input [6]: [cp_catalog_page_id#57, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Keys [1]: [cp_catalog_page_id#57] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#49)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#50 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#54 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#49))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(CheckOverflow((promote_precision(cast(cs_net_profit#50 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#54 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#72] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#49))#70,17,2) AS sales#73, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 AS returns#74, sum(CheckOverflow((promote_precision(cast(cs_net_profit#50 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#54 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#72 AS profit#75, catalog channel AS channel#76, concat(catalog_page, cp_catalog_page_id#57) AS id#77] - -(67) Scan parquet default.web_sales -Output [7]: [ws_sold_date_sk#78, ws_item_sk#79, ws_web_site_sk#80, ws_promo_sk#81, ws_order_number#82, ws_ext_sales_price#83, ws_net_profit#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(68) ColumnarToRow [codegen id : 20] -Input [7]: [ws_sold_date_sk#78, ws_item_sk#79, ws_web_site_sk#80, ws_promo_sk#81, ws_order_number#82, ws_ext_sales_price#83, ws_net_profit#84] - -(69) Filter [codegen id : 20] -Input [7]: [ws_sold_date_sk#78, ws_item_sk#79, ws_web_site_sk#80, ws_promo_sk#81, ws_order_number#82, ws_ext_sales_price#83, ws_net_profit#84] -Condition : (((isnotnull(ws_sold_date_sk#78) AND isnotnull(ws_web_site_sk#80)) AND isnotnull(ws_item_sk#79)) AND isnotnull(ws_promo_sk#81)) - -(70) Scan parquet default.web_returns -Output [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 15] -Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] - -(72) Filter [codegen id : 15] -Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Condition : (isnotnull(wr_item_sk#85) AND isnotnull(wr_order_number#86)) - -(73) BroadcastExchange -Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[1, bigint, false]),false), [id=#89] - -(74) BroadcastHashJoin [codegen id : 20] -Left keys [2]: [cast(ws_item_sk#79 as bigint), cast(ws_order_number#82 as bigint)] -Right keys [2]: [wr_item_sk#85, wr_order_number#86] -Join condition: None - -(75) Project [codegen id : 20] -Output [8]: [ws_sold_date_sk#78, ws_item_sk#79, ws_web_site_sk#80, ws_promo_sk#81, ws_ext_sales_price#83, ws_net_profit#84, wr_return_amt#87, wr_net_loss#88] -Input [11]: [ws_sold_date_sk#78, ws_item_sk#79, ws_web_site_sk#80, ws_promo_sk#81, ws_order_number#82, ws_ext_sales_price#83, ws_net_profit#84, wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] - -(76) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#13] - -(77) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ws_sold_date_sk#78] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(78) Project [codegen id : 20] -Output [7]: [ws_item_sk#79, ws_web_site_sk#80, ws_promo_sk#81, ws_ext_sales_price#83, ws_net_profit#84, wr_return_amt#87, wr_net_loss#88] -Input [9]: [ws_sold_date_sk#78, ws_item_sk#79, ws_web_site_sk#80, ws_promo_sk#81, ws_ext_sales_price#83, ws_net_profit#84, wr_return_amt#87, wr_net_loss#88, d_date_sk#13] - -(79) Scan parquet default.web_site -Output [2]: [web_site_sk#90, web_site_id#91] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(80) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#90, web_site_id#91] - -(81) Filter [codegen id : 17] -Input [2]: [web_site_sk#90, web_site_id#91] -Condition : isnotnull(web_site_sk#90) - -(82) BroadcastExchange -Input [2]: [web_site_sk#90, web_site_id#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] - -(83) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ws_web_site_sk#80] -Right keys [1]: [web_site_sk#90] -Join condition: None - -(84) Project [codegen id : 20] -Output [7]: [ws_item_sk#79, ws_promo_sk#81, ws_ext_sales_price#83, ws_net_profit#84, wr_return_amt#87, wr_net_loss#88, web_site_id#91] -Input [9]: [ws_item_sk#79, ws_web_site_sk#80, ws_promo_sk#81, ws_ext_sales_price#83, ws_net_profit#84, wr_return_amt#87, wr_net_loss#88, web_site_sk#90, web_site_id#91] - -(85) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#19] - -(86) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ws_item_sk#79] -Right keys [1]: [i_item_sk#19] -Join condition: None - -(87) Project [codegen id : 20] -Output [6]: [ws_promo_sk#81, ws_ext_sales_price#83, ws_net_profit#84, wr_return_amt#87, wr_net_loss#88, web_site_id#91] -Input [8]: [ws_item_sk#79, ws_promo_sk#81, ws_ext_sales_price#83, ws_net_profit#84, wr_return_amt#87, wr_net_loss#88, web_site_id#91, i_item_sk#19] - -(88) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#22] - -(89) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ws_promo_sk#81] -Right keys [1]: [p_promo_sk#22] -Join condition: None - -(90) Project [codegen id : 20] -Output [5]: [ws_ext_sales_price#83, ws_net_profit#84, wr_return_amt#87, wr_net_loss#88, web_site_id#91] -Input [7]: [ws_promo_sk#81, ws_ext_sales_price#83, ws_net_profit#84, wr_return_amt#87, wr_net_loss#88, web_site_id#91, p_promo_sk#22] - -(91) HashAggregate [codegen id : 20] -Input [5]: [ws_ext_sales_price#83, ws_net_profit#84, wr_return_amt#87, wr_net_loss#88, web_site_id#91] -Keys [1]: [web_site_id#91] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#83)), partial_sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#84 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [6]: [web_site_id#91, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] - -(92) Exchange -Input [6]: [web_site_id#91, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#91, 5), true, [id=#103] - -(93) HashAggregate [codegen id : 21] -Input [6]: [web_site_id#91, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Keys [1]: [web_site_id#91] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#83)), sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#84 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#83))#104, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#105, sum(CheckOverflow((promote_precision(cast(ws_net_profit#84 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#106] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#83))#104,17,2) AS sales#107, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#105 AS returns#108, sum(CheckOverflow((promote_precision(cast(ws_net_profit#84 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#106 AS profit#109, web channel AS channel#110, concat(web_site, web_site_id#91) AS id#111] - -(94) Union - -(95) Expand [codegen id : 22] -Input [5]: [sales#39, returns#40, profit#41, channel#42, id#43] -Arguments: [List(sales#39, returns#40, profit#41, channel#42, id#43, 0), List(sales#39, returns#40, profit#41, channel#42, null, 1), List(sales#39, returns#40, profit#41, null, null, 3)], [sales#39, returns#40, profit#41, channel#112, id#113, spark_grouping_id#114] - -(96) HashAggregate [codegen id : 22] -Input [6]: [sales#39, returns#40, profit#41, channel#112, id#113, spark_grouping_id#114] -Keys [3]: [channel#112, id#113, spark_grouping_id#114] -Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] -Aggregate Attributes [6]: [sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] -Results [9]: [channel#112, id#113, spark_grouping_id#114, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] - -(97) Exchange -Input [9]: [channel#112, id#113, spark_grouping_id#114, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Arguments: hashpartitioning(channel#112, id#113, spark_grouping_id#114, 5), true, [id=#127] - -(98) HashAggregate [codegen id : 23] -Input [9]: [channel#112, id#113, spark_grouping_id#114, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Keys [3]: [channel#112, id#113, spark_grouping_id#114] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#128, sum(returns#40)#129, sum(profit#41)#130] -Results [5]: [channel#112, id#113, sum(sales#39)#128 AS sales#131, sum(returns#40)#129 AS returns#132, sum(profit#41)#130 AS profit#133] - -(99) TakeOrderedAndProject -Input [5]: [channel#112, id#113, sales#131, returns#132, profit#133] -Arguments: 100, [channel#112 ASC NULLS FIRST, id#113 ASC NULLS FIRST], [channel#112, id#113, sales#131, returns#132, profit#133] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q80/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q80/simplified.txt deleted file mode 100644 index 37f46f14e..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q80/simplified.txt +++ /dev/null @@ -1,148 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (23) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (22) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (7) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [s_store_id] #2 - WholeStageCodegen (6) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Project [i_item_sk] - Filter [i_current_price,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Project [p_promo_sk] - Filter [p_channel_tv,p_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (14) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cp_catalog_page_id] #8 - WholeStageCodegen (13) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - Filter [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Filter [cr_item_sk,cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (10) - Filter [cp_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #6 - InputAdapter - ReusedExchange [p_promo_sk] #7 - WholeStageCodegen (21) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [web_site_id] #11 - WholeStageCodegen (20) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - Filter [ws_sold_date_sk,ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (15) - Filter [wr_item_sk,wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (17) - Filter [web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #6 - InputAdapter - ReusedExchange [p_promo_sk] #7 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q81/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q81/explain.txt deleted file mode 100644 index 2530fe9e8..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q81/explain.txt +++ /dev/null @@ -1,298 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (52) -+- * Project (51) - +- * BroadcastHashJoin Inner BuildRight (50) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (20) - : : : +- * HashAggregate (19) - : : : +- Exchange (18) - : : : +- * HashAggregate (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.catalog_returns (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.customer_address (11) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- Exchange (34) - : : +- * HashAggregate (33) - : : +- * HashAggregate (32) - : : +- Exchange (31) - : : +- * HashAggregate (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (26) - : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : :- * Filter (23) - : : : : +- * ColumnarToRow (22) - : : : : +- Scan parquet default.catalog_returns (21) - : : : +- ReusedExchange (24) - : : +- ReusedExchange (27) - : +- BroadcastExchange (43) - : +- * Filter (42) - : +- * ColumnarToRow (41) - : +- Scan parquet default.customer (40) - +- BroadcastExchange (49) - +- * Filter (48) - +- * ColumnarToRow (47) - +- Scan parquet default.customer_address (46) - - -(1) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] - -(3) Filter [codegen id : 3] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Condition : ((isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) AND isnotnull(cr_returning_customer_sk#2)) - -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_year#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_year#6] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_year#6] -Condition : ((isnotnull(d_year#6) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_year#6] - -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returned_date_sk#1] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] - -(11) Scan parquet default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#8, ca_state#9] - -(13) Filter [codegen id : 2] -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_state#9)) - -(14) BroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returning_addr_sk#3] -Right keys [1]: [ca_address_sk#8] -Join condition: None - -(16) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#9] -Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#8, ca_state#9] - -(17) HashAggregate [codegen id : 3] -Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#9] -Keys [2]: [cr_returning_customer_sk#2, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum#11] -Results [3]: [cr_returning_customer_sk#2, ca_state#9, sum#12] - -(18) Exchange -Input [3]: [cr_returning_customer_sk#2, ca_state#9, sum#12] -Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#9, 5), true, [id=#13] - -(19) HashAggregate [codegen id : 11] -Input [3]: [cr_returning_customer_sk#2, ca_state#9, sum#12] -Keys [2]: [cr_returning_customer_sk#2, ca_state#9] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#14] -Results [3]: [cr_returning_customer_sk#2 AS ctr_customer_sk#15, ca_state#9 AS ctr_state#16, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#14,17,2) AS ctr_total_return#17] - -(20) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#15, ctr_state#16, ctr_total_return#17] -Condition : isnotnull(ctr_total_return#17) - -(21) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct - -(22) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] - -(23) Filter [codegen id : 6] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Condition : (isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) - -(24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#1] -Right keys [1]: [d_date_sk#5] -Join condition: None - -(26) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] - -(27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#8, ca_state#9] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#3] -Right keys [1]: [ca_address_sk#8] -Join condition: None - -(29) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#9] -Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#8, ca_state#9] - -(30) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#9] -Keys [2]: [cr_returning_customer_sk#2, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [cr_returning_customer_sk#2, ca_state#9, sum#19] - -(31) Exchange -Input [3]: [cr_returning_customer_sk#2, ca_state#9, sum#19] -Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#9, 5), true, [id=#20] - -(32) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#2, ca_state#9, sum#19] -Keys [2]: [cr_returning_customer_sk#2, ca_state#9] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#21] -Results [2]: [ca_state#9 AS ctr_state#16, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#21,17,2) AS ctr_total_return#17] - -(33) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#16, ctr_total_return#17] -Keys [1]: [ctr_state#16] -Functions [1]: [partial_avg(ctr_total_return#17)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [3]: [ctr_state#16, sum#24, count#25] - -(34) Exchange -Input [3]: [ctr_state#16, sum#24, count#25] -Arguments: hashpartitioning(ctr_state#16, 5), true, [id=#26] - -(35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#16, sum#24, count#25] -Keys [1]: [ctr_state#16] -Functions [1]: [avg(ctr_total_return#17)] -Aggregate Attributes [1]: [avg(ctr_total_return#17)#27] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#17)#27) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28, ctr_state#16 AS ctr_state#16#29] - -(36) Filter [codegen id : 8] -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28, ctr_state#16#29] -Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28) - -(37) BroadcastExchange -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28, ctr_state#16#29] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#30] - -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#16] -Right keys [1]: [ctr_state#16#29] -Join condition: (cast(ctr_total_return#17 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28) - -(39) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#15, ctr_total_return#17] -Input [5]: [ctr_customer_sk#15, ctr_state#16, ctr_total_return#17, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#28, ctr_state#16#29] - -(40) Scan parquet default.customer -Output [6]: [c_customer_sk#31, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#31, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36] - -(42) Filter [codegen id : 9] -Input [6]: [c_customer_sk#31, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36] -Condition : (isnotnull(c_customer_sk#31) AND isnotnull(c_current_addr_sk#33)) - -(43) BroadcastExchange -Input [6]: [c_customer_sk#31, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] - -(44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#15] -Right keys [1]: [c_customer_sk#31] -Join condition: None - -(45) Project [codegen id : 11] -Output [6]: [ctr_total_return#17, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36] -Input [8]: [ctr_customer_sk#15, ctr_total_return#17, c_customer_sk#31, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36] - -(46) Scan parquet default.customer_address -Output [12]: [ca_address_sk#8, ca_street_number#38, ca_street_name#39, ca_street_type#40, ca_suite_number#41, ca_city#42, ca_county#43, ca_state#9, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#8, ca_street_number#38, ca_street_name#39, ca_street_type#40, ca_suite_number#41, ca_city#42, ca_county#43, ca_state#9, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47] - -(48) Filter [codegen id : 10] -Input [12]: [ca_address_sk#8, ca_street_number#38, ca_street_name#39, ca_street_type#40, ca_suite_number#41, ca_city#42, ca_county#43, ca_state#9, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47] -Condition : ((isnotnull(ca_state#9) AND (ca_state#9 = GA)) AND isnotnull(ca_address_sk#8)) - -(49) BroadcastExchange -Input [12]: [ca_address_sk#8, ca_street_number#38, ca_street_name#39, ca_street_type#40, ca_suite_number#41, ca_city#42, ca_county#43, ca_state#9, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] - -(50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#8] -Join condition: None - -(51) Project [codegen id : 11] -Output [16]: [c_customer_id#32, c_salutation#34, c_first_name#35, c_last_name#36, ca_street_number#38, ca_street_name#39, ca_street_type#40, ca_suite_number#41, ca_city#42, ca_county#43, ca_state#9, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#17] -Input [18]: [ctr_total_return#17, c_customer_id#32, c_current_addr_sk#33, c_salutation#34, c_first_name#35, c_last_name#36, ca_address_sk#8, ca_street_number#38, ca_street_name#39, ca_street_type#40, ca_suite_number#41, ca_city#42, ca_county#43, ca_state#9, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47] - -(52) TakeOrderedAndProject -Input [16]: [c_customer_id#32, c_salutation#34, c_first_name#35, c_last_name#36, ca_street_number#38, ca_street_name#39, ca_street_type#40, ca_suite_number#41, ca_city#42, ca_county#43, ca_state#9, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#17] -Arguments: 100, [c_customer_id#32 ASC NULLS FIRST, c_salutation#34 ASC NULLS FIRST, c_first_name#35 ASC NULLS FIRST, c_last_name#36 ASC NULLS FIRST, ca_street_number#38 ASC NULLS FIRST, ca_street_name#39 ASC NULLS FIRST, ca_street_type#40 ASC NULLS FIRST, ca_suite_number#41 ASC NULLS FIRST, ca_city#42 ASC NULLS FIRST, ca_county#43 ASC NULLS FIRST, ca_state#9 ASC NULLS FIRST, ca_zip#44 ASC NULLS FIRST, ca_country#45 ASC NULLS FIRST, ca_gmt_offset#46 ASC NULLS FIRST, ca_location_type#47 ASC NULLS FIRST, ctr_total_return#17 ASC NULLS FIRST], [c_customer_id#32, c_salutation#34, c_first_name#35, c_last_name#36, ca_street_number#38, ca_street_name#39, ca_street_type#40, ca_suite_number#41, ca_city#42, ca_county#43, ca_state#9, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#17] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q81/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q81/simplified.txt deleted file mode 100644 index 2b7c6041b..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q81/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - WholeStageCodegen (11) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) - Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] - InputAdapter - Exchange [ctr_state] #5 - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returned_date_sk,cr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q82/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q82/explain.txt deleted file mode 100644 index c7240792c..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q82/explain.txt +++ /dev/null @@ -1,160 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * BroadcastHashJoin Inner BuildRight (23) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.item (1) - : : +- BroadcastExchange (9) - : : +- * Project (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet default.inventory (5) - : +- BroadcastExchange (16) - : +- * Project (15) - : +- * Filter (14) - : +- * ColumnarToRow (13) - : +- Scan parquet default.date_dim (12) - +- BroadcastExchange (22) - +- * Filter (21) - +- * ColumnarToRow (20) - +- Scan parquet default.store_sales (19) - - -(1) Scan parquet default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), In(i_manufact_id, [129,270,821,423]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] - -(3) Filter [codegen id : 4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (cast(i_current_price#4 as decimal(12,2)) <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) - -(4) Project [codegen id : 4] -Output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] - -(5) Scan parquet default.inventory -Output [3]: [inv_date_sk#6, inv_item_sk#7, inv_quantity_on_hand#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk), IsNotNull(inv_date_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [inv_date_sk#6, inv_item_sk#7, inv_quantity_on_hand#8] - -(7) Filter [codegen id : 1] -Input [3]: [inv_date_sk#6, inv_item_sk#7, inv_quantity_on_hand#8] -Condition : ((((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) AND isnotnull(inv_date_sk#6)) - -(8) Project [codegen id : 1] -Output [2]: [inv_date_sk#6, inv_item_sk#7] -Input [3]: [inv_date_sk#6, inv_item_sk#7, inv_quantity_on_hand#8] - -(9) BroadcastExchange -Input [2]: [inv_date_sk#6, inv_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#9] - -(10) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#7] -Join condition: None - -(11) Project [codegen id : 4] -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#6] -Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#6, inv_item_sk#7] - -(12) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_date#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#10, d_date#11] - -(14) Filter [codegen id : 2] -Input [2]: [d_date_sk#10, d_date#11] -Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 11102)) AND (d_date#11 <= 11162)) AND isnotnull(d_date_sk#10)) - -(15) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_date#11] - -(16) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#6] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(18) Project [codegen id : 4] -Output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#6, d_date_sk#10] - -(19) Scan parquet default.store_sales -Output [1]: [ss_item_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 3] -Input [1]: [ss_item_sk#13] - -(21) Filter [codegen id : 3] -Input [1]: [ss_item_sk#13] -Condition : isnotnull(ss_item_sk#13) - -(22) BroadcastExchange -Input [1]: [ss_item_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#13] -Join condition: None - -(24) Project [codegen id : 4] -Output [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, ss_item_sk#13] - -(25) HashAggregate [codegen id : 4] -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] - -(26) Exchange -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), true, [id=#15] - -(27) HashAggregate [codegen id : 5] -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] - -(28) TakeOrderedAndProject -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q82/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q82/simplified.txt deleted file mode 100644 index 92f8729ca..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q82/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - WholeStageCodegen (5) - HashAggregate [i_item_id,i_item_desc,i_current_price] - InputAdapter - Exchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - Filter [i_current_price,i_manufact_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [inv_date_sk,inv_item_sk] - Filter [inv_quantity_on_hand,inv_item_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q83/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q83/explain.txt deleted file mode 100644 index 0d44a0146..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q83/explain.txt +++ /dev/null @@ -1,344 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * HashAggregate (30) - : : +- Exchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_returns (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- * BroadcastHashJoin LeftSemi BuildRight (20) - : : :- * ColumnarToRow (14) - : : : +- Scan parquet default.date_dim (13) - : : +- BroadcastExchange (19) - : : +- * Project (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.date_dim (15) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Filter (33) - : : : +- * ColumnarToRow (32) - : : : +- Scan parquet default.catalog_returns (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) - - -(1) Scan parquet default.store_returns -Output [3]: [sr_returned_date_sk#1, sr_item_sk#2, sr_return_quantity#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [3]: [sr_returned_date_sk#1, sr_item_sk#2, sr_return_quantity#3] - -(3) Filter [codegen id : 5] -Input [3]: [sr_returned_date_sk#1, sr_item_sk#2, sr_return_quantity#3] -Condition : (isnotnull(sr_item_sk#2) AND isnotnull(sr_returned_date_sk#1)) - -(4) Scan parquet default.item -Output [2]: [i_item_sk#4, i_item_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] - -(6) Filter [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] -Condition : (isnotnull(i_item_sk#4) AND isnotnull(i_item_id#5)) - -(7) BroadcastExchange -Input [2]: [i_item_sk#4, i_item_id#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_item_sk#2] -Right keys [1]: [cast(i_item_sk#4 as bigint)] -Join condition: None - -(9) Project [codegen id : 5] -Output [3]: [sr_returned_date_sk#1, sr_return_quantity#3, i_item_id#5] -Input [5]: [sr_returned_date_sk#1, sr_item_sk#2, sr_return_quantity#3, i_item_sk#4, i_item_id#5] - -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#7, d_date#8] - -(12) Filter [codegen id : 4] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) - -(13) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [d_date#8, d_week_seq#9] - -(15) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(16) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] - -(17) Filter [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] -Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) - -(18) Project [codegen id : 2] -Output [1]: [d_week_seq#9 AS d_week_seq#9#10] -Input [2]: [d_date#8, d_week_seq#9] - -(19) BroadcastExchange -Input [1]: [d_week_seq#9#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#9] -Right keys [1]: [d_week_seq#9#10] -Join condition: None - -(21) Project [codegen id : 3] -Output [1]: [d_date#8 AS d_date#8#12] -Input [2]: [d_date#8, d_week_seq#9] - -(22) BroadcastExchange -Input [1]: [d_date#8#12] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#12] -Join condition: None - -(24) Project [codegen id : 4] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] - -(25) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_returned_date_sk#1] -Right keys [1]: [cast(d_date_sk#7 as bigint)] -Join condition: None - -(27) Project [codegen id : 5] -Output [2]: [sr_return_quantity#3, i_item_id#5] -Input [4]: [sr_returned_date_sk#1, sr_return_quantity#3, i_item_id#5, d_date_sk#7] - -(28) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#3, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(cast(sr_return_quantity#3 as bigint))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#5, sum#16] - -(29) Exchange -Input [2]: [i_item_id#5, sum#16] -Arguments: hashpartitioning(i_item_id#5, 5), true, [id=#17] - -(30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#5, sum#16] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(cast(sr_return_quantity#3 as bigint))] -Aggregate Attributes [1]: [sum(cast(sr_return_quantity#3 as bigint))#18] -Results [2]: [i_item_id#5 AS item_id#19, sum(cast(sr_return_quantity#3 as bigint))#18 AS sr_item_qty#20] - -(31) Scan parquet default.catalog_returns -Output [3]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_returned_date_sk)] -ReadSchema: struct - -(32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23] - -(33) Filter [codegen id : 10] -Input [3]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23] -Condition : (isnotnull(cr_item_sk#22) AND isnotnull(cr_returned_date_sk#21)) - -(34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] - -(35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#4] -Join condition: None - -(36) Project [codegen id : 10] -Output [3]: [cr_returned_date_sk#21, cr_return_quantity#23, i_item_id#5] -Input [5]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23, i_item_sk#4, i_item_id#5] - -(37) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#7] - -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#21] -Right keys [1]: [d_date_sk#7] -Join condition: None - -(39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#5] -Input [4]: [cr_returned_date_sk#21, cr_return_quantity#23, i_item_id#5, d_date_sk#7] - -(40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(cast(cr_return_quantity#23 as bigint))] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#5, sum#25] - -(41) Exchange -Input [2]: [i_item_id#5, sum#25] -Arguments: hashpartitioning(i_item_id#5, 5), true, [id=#26] - -(42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#5, sum#25] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(cast(cr_return_quantity#23 as bigint))] -Aggregate Attributes [1]: [sum(cast(cr_return_quantity#23 as bigint))#27] -Results [2]: [i_item_id#5 AS item_id#28, sum(cast(cr_return_quantity#23 as bigint))#27 AS cr_item_qty#29] - -(43) BroadcastExchange -Input [2]: [item_id#28, cr_item_qty#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] - -(44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#28] -Join condition: None - -(45) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] -Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] - -(46) Scan parquet default.web_returns -Output [3]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_returned_date_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33] - -(48) Filter [codegen id : 16] -Input [3]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33] -Condition : (isnotnull(wr_item_sk#32) AND isnotnull(wr_returned_date_sk#31)) - -(49) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] - -(50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#32] -Right keys [1]: [cast(i_item_sk#4 as bigint)] -Join condition: None - -(51) Project [codegen id : 16] -Output [3]: [wr_returned_date_sk#31, wr_return_quantity#33, i_item_id#5] -Input [5]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33, i_item_sk#4, i_item_id#5] - -(52) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#7] - -(53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#31] -Right keys [1]: [cast(d_date_sk#7 as bigint)] -Join condition: None - -(54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#33, i_item_id#5] -Input [4]: [wr_returned_date_sk#31, wr_return_quantity#33, i_item_id#5, d_date_sk#7] - -(55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#33, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(cast(wr_return_quantity#33 as bigint))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#5, sum#35] - -(56) Exchange -Input [2]: [i_item_id#5, sum#35] -Arguments: hashpartitioning(i_item_id#5, 5), true, [id=#36] - -(57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#5, sum#35] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(cast(wr_return_quantity#33 as bigint))] -Aggregate Attributes [1]: [sum(cast(wr_return_quantity#33 as bigint))#37] -Results [2]: [i_item_id#5 AS item_id#38, sum(cast(wr_return_quantity#33 as bigint))#37 AS wr_item_qty#39] - -(58) BroadcastExchange -Input [2]: [item_id#38, wr_item_qty#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] - -(59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#38] -Join condition: None - -(60) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] - -(61) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q83/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q83/simplified.txt deleted file mode 100644 index c12d5ffaa..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q83/simplified.txt +++ /dev/null @@ -1,91 +0,0 @@ -TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - WholeStageCodegen (18) - Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] - BroadcastHashJoin [item_id,item_id] - Project [item_id,sr_item_qty,cr_item_qty] - BroadcastHashJoin [item_id,item_id] - HashAggregate [i_item_id,sum] [sum(cast(sr_return_quantity as bigint)),item_id,sr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - Project [sr_return_quantity,i_item_id] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [sr_returned_date_sk,sr_return_quantity,i_item_id] - BroadcastHashJoin [sr_item_sk,i_item_sk] - Filter [sr_item_sk,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_return_quantity] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [i_item_sk,i_item_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (4) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [d_week_seq] - Filter [d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (11) - HashAggregate [i_item_id,sum] [sum(cast(cr_return_quantity as bigint)),item_id,cr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (10) - HashAggregate [i_item_id,cr_return_quantity] [sum,sum] - Project [cr_return_quantity,i_item_id] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cr_returned_date_sk,cr_return_quantity,i_item_id] - BroadcastHashJoin [cr_item_sk,i_item_sk] - Filter [cr_item_sk,cr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_item_sk,cr_return_quantity] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,sum] [sum(cast(wr_return_quantity as bigint)),item_id,wr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #9 - WholeStageCodegen (16) - HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - Project [wr_return_quantity,i_item_id] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Project [wr_returned_date_sk,wr_return_quantity,i_item_id] - BroadcastHashJoin [wr_item_sk,i_item_sk] - Filter [wr_item_sk,wr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_returned_date_sk,wr_item_sk,wr_return_quantity] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q84/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q84/explain.txt deleted file mode 100644 index 9e5fb4386..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q84/explain.txt +++ /dev/null @@ -1,200 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (36) -+- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.customer_address (4) - : : : +- BroadcastExchange (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.customer_demographics (11) - : : +- BroadcastExchange (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.household_demographics (17) - : +- BroadcastExchange (27) - : +- * Project (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.income_band (23) - +- BroadcastExchange (33) - +- * Filter (32) - +- * ColumnarToRow (31) - +- Scan parquet default.store_returns (30) - - -(1) Scan parquet default.customer -Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] - -(3) Filter [codegen id : 6] -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) - -(4) Scan parquet default.customer_address -Output [2]: [ca_address_sk#7, ca_city#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [ca_address_sk#7, ca_city#8] - -(6) Filter [codegen id : 1] -Input [2]: [ca_address_sk#7, ca_city#8] -Condition : ((isnotnull(ca_city#8) AND (ca_city#8 = Edgewood)) AND isnotnull(ca_address_sk#7)) - -(7) Project [codegen id : 1] -Output [1]: [ca_address_sk#7] -Input [2]: [ca_address_sk#7, ca_city#8] - -(8) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] - -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_addr_sk#4] -Right keys [1]: [ca_address_sk#7] -Join condition: None - -(10) Project [codegen id : 6] -Output [5]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6] -Input [7]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6, ca_address_sk#7] - -(11) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [1]: [cd_demo_sk#10] - -(13) Filter [codegen id : 2] -Input [1]: [cd_demo_sk#10] -Condition : isnotnull(cd_demo_sk#10) - -(14) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join condition: None - -(16) Project [codegen id : 6] -Output [5]: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#10] -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#10] - -(17) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#12, hd_income_band_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [2]: [hd_demo_sk#12, hd_income_band_sk#13] - -(19) Filter [codegen id : 3] -Input [2]: [hd_demo_sk#12, hd_income_band_sk#13] -Condition : (isnotnull(hd_demo_sk#12) AND isnotnull(hd_income_band_sk#13)) - -(20) BroadcastExchange -Input [2]: [hd_demo_sk#12, hd_income_band_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#12] -Join condition: None - -(22) Project [codegen id : 6] -Output [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#10, hd_income_band_sk#13] -Input [7]: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#10, hd_demo_sk#12, hd_income_band_sk#13] - -(23) Scan parquet default.income_band -Output [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] - -(25) Filter [codegen id : 4] -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Condition : ((((isnotnull(ib_lower_bound#16) AND isnotnull(ib_upper_bound#17)) AND (ib_lower_bound#16 >= 38128)) AND (ib_upper_bound#17 <= 88128)) AND isnotnull(ib_income_band_sk#15)) - -(26) Project [codegen id : 4] -Output [1]: [ib_income_band_sk#15] -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] - -(27) BroadcastExchange -Input [1]: [ib_income_band_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [hd_income_band_sk#13] -Right keys [1]: [ib_income_band_sk#15] -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#10] -Input [6]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#10, hd_income_band_sk#13, ib_income_band_sk#15] - -(30) Scan parquet default.store_returns -Output [1]: [sr_cdemo_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_cdemo_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 5] -Input [1]: [sr_cdemo_sk#19] - -(32) Filter [codegen id : 5] -Input [1]: [sr_cdemo_sk#19] -Condition : isnotnull(sr_cdemo_sk#19) - -(33) BroadcastExchange -Input [1]: [sr_cdemo_sk#19] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [id=#20] - -(34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cast(cd_demo_sk#10 as bigint)] -Right keys [1]: [sr_cdemo_sk#19] -Join condition: None - -(35) Project [codegen id : 6] -Output [3]: [c_customer_id#1 AS customer_id#21, concat(c_last_name#6, , , c_first_name#5) AS customername#22, c_customer_id#1] -Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#10, sr_cdemo_sk#19] - -(36) TakeOrderedAndProject -Input [3]: [customer_id#21, customername#22, c_customer_id#1] -Arguments: 100, [c_customer_id#1 ASC NULLS FIRST], [customer_id#21, customername#22] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q84/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q84/simplified.txt deleted file mode 100644 index 014d808cf..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q84/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject [c_customer_id,customer_id,customername] - WholeStageCodegen (6) - Project [c_customer_id,c_last_name,c_first_name] - BroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - Project [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #1 - WholeStageCodegen (1) - Project [ca_address_sk] - Filter [ca_city,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Filter [hd_demo_sk,hd_income_band_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_income_band_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ib_income_band_sk] - Filter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] - ColumnarToRow - InputAdapter - Scan parquet default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - Filter [sr_cdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_cdemo_sk] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q85/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q85/explain.txt deleted file mode 100644 index 23598c2e7..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q85/explain.txt +++ /dev/null @@ -1,287 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Project (27) - : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : :- * Project (21) - : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.web_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet default.web_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet default.web_page (10) - : : : : +- BroadcastExchange (19) - : : : : +- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.customer_demographics (16) - : : : +- BroadcastExchange (25) - : : : +- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.customer_demographics (22) - : : +- BroadcastExchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.customer_address (28) - : +- BroadcastExchange (39) - : +- * Project (38) - : +- * Filter (37) - : +- * ColumnarToRow (36) - : +- Scan parquet default.date_dim (35) - +- BroadcastExchange (45) - +- * Filter (44) - +- * ColumnarToRow (43) - +- Scan parquet default.reason (42) - - -(1) Scan parquet default.web_sales -Output [7]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), IsNotNull(ws_sold_date_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [7]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] - -(3) Filter [codegen id : 8] -Input [7]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] -Condition : (((((isnotnull(ws_item_sk#2) AND isnotnull(ws_order_number#4)) AND isnotnull(ws_web_page_sk#3)) AND isnotnull(ws_sold_date_sk#1)) AND ((((ws_sales_price#6 >= 100.00) AND (ws_sales_price#6 <= 150.00)) OR ((ws_sales_price#6 >= 50.00) AND (ws_sales_price#6 <= 100.00))) OR ((ws_sales_price#6 >= 150.00) AND (ws_sales_price#6 <= 200.00)))) AND ((((ws_net_profit#7 >= 100.00) AND (ws_net_profit#7 <= 200.00)) OR ((ws_net_profit#7 >= 150.00) AND (ws_net_profit#7 <= 300.00))) OR ((ws_net_profit#7 >= 50.00) AND (ws_net_profit#7 <= 250.00)))) - -(4) Scan parquet default.web_returns -Output [8]: [wr_item_sk#8, wr_refunded_cdemo_sk#9, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_order_number#13, wr_fee#14, wr_refunded_cash#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [8]: [wr_item_sk#8, wr_refunded_cdemo_sk#9, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_order_number#13, wr_fee#14, wr_refunded_cash#15] - -(6) Filter [codegen id : 1] -Input [8]: [wr_item_sk#8, wr_refunded_cdemo_sk#9, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_order_number#13, wr_fee#14, wr_refunded_cash#15] -Condition : (((((isnotnull(wr_item_sk#8) AND isnotnull(wr_order_number#13)) AND isnotnull(wr_refunded_cdemo_sk#9)) AND isnotnull(wr_returning_cdemo_sk#11)) AND isnotnull(wr_refunded_addr_sk#10)) AND isnotnull(wr_reason_sk#12)) - -(7) BroadcastExchange -Input [8]: [wr_item_sk#8, wr_refunded_cdemo_sk#9, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_order_number#13, wr_fee#14, wr_refunded_cash#15] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[5, bigint, false]),false), [id=#16] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [cast(ws_item_sk#2 as bigint), cast(ws_order_number#4 as bigint)] -Right keys [2]: [wr_item_sk#8, wr_order_number#13] -Join condition: None - -(9) Project [codegen id : 8] -Output [11]: [ws_sold_date_sk#1, ws_web_page_sk#3, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#9, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15] -Input [15]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_item_sk#8, wr_refunded_cdemo_sk#9, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_order_number#13, wr_fee#14, wr_refunded_cash#15] - -(10) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [1]: [wp_web_page_sk#17] - -(12) Filter [codegen id : 2] -Input [1]: [wp_web_page_sk#17] -Condition : isnotnull(wp_web_page_sk#17) - -(13) BroadcastExchange -Input [1]: [wp_web_page_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#17] -Join condition: None - -(15) Project [codegen id : 8] -Output [10]: [ws_sold_date_sk#1, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#9, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15] -Input [12]: [ws_sold_date_sk#1, ws_web_page_sk#3, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#9, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15, wp_web_page_sk#17] - -(16) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -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)))] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] - -(18) Filter [codegen id : 3] -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(cd_marital_status#20)) AND isnotnull(cd_education_status#21)) AND ((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree)) OR ((cd_marital_status#20 = S) AND (cd_education_status#21 = College))) OR ((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree)))) - -(19) BroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_refunded_cdemo_sk#9] -Right keys [1]: [cast(cd_demo_sk#19 as bigint)] -Join condition: ((((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree)) AND (ws_sales_price#6 >= 100.00)) AND (ws_sales_price#6 <= 150.00)) OR ((((cd_marital_status#20 = S) AND (cd_education_status#21 = College)) AND (ws_sales_price#6 >= 50.00)) AND (ws_sales_price#6 <= 100.00))) OR ((((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree)) AND (ws_sales_price#6 >= 150.00)) AND (ws_sales_price#6 <= 200.00))) - -(21) Project [codegen id : 8] -Output [10]: [ws_sold_date_sk#1, ws_quantity#5, ws_net_profit#7, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15, cd_marital_status#20, cd_education_status#21] -Input [13]: [ws_sold_date_sk#1, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#9, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15, cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] - -(22) Scan parquet default.customer_demographics -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: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] -ReadSchema: struct - -(23) ColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] - -(24) Filter [codegen id : 4] -Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Condition : ((isnotnull(cd_demo_sk#23) AND isnotnull(cd_marital_status#24)) AND isnotnull(cd_education_status#25)) - -(25) BroadcastExchange -Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), input[1, string, false], input[2, string, false]),false), [id=#26] - -(26) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [wr_returning_cdemo_sk#11, cd_marital_status#20, cd_education_status#21] -Right keys [3]: [cast(cd_demo_sk#23 as bigint), cd_marital_status#24, cd_education_status#25] -Join condition: None - -(27) Project [codegen id : 8] -Output [7]: [ws_sold_date_sk#1, ws_quantity#5, ws_net_profit#7, wr_refunded_addr_sk#10, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15] -Input [13]: [ws_sold_date_sk#1, ws_quantity#5, ws_net_profit#7, wr_refunded_addr_sk#10, wr_returning_cdemo_sk#11, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15, cd_marital_status#20, cd_education_status#21, cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] - -(28) Scan parquet default.customer_address -Output [3]: [ca_address_sk#27, ca_state#28, ca_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [IN,OH,NJ]),In(ca_state, [WI,CT,KY])),In(ca_state, [LA,IA,AR]))] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 5] -Input [3]: [ca_address_sk#27, ca_state#28, ca_country#29] - -(30) Filter [codegen id : 5] -Input [3]: [ca_address_sk#27, ca_state#28, ca_country#29] -Condition : (((isnotnull(ca_country#29) AND (ca_country#29 = United States)) AND isnotnull(ca_address_sk#27)) AND ((ca_state#28 IN (IN,OH,NJ) OR ca_state#28 IN (WI,CT,KY)) OR ca_state#28 IN (LA,IA,AR))) - -(31) Project [codegen id : 5] -Output [2]: [ca_address_sk#27, ca_state#28] -Input [3]: [ca_address_sk#27, ca_state#28, ca_country#29] - -(32) BroadcastExchange -Input [2]: [ca_address_sk#27, ca_state#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] - -(33) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_refunded_addr_sk#10] -Right keys [1]: [cast(ca_address_sk#27 as bigint)] -Join condition: ((((ca_state#28 IN (IN,OH,NJ) AND (ws_net_profit#7 >= 100.00)) AND (ws_net_profit#7 <= 200.00)) OR ((ca_state#28 IN (WI,CT,KY) AND (ws_net_profit#7 >= 150.00)) AND (ws_net_profit#7 <= 300.00))) OR ((ca_state#28 IN (LA,IA,AR) AND (ws_net_profit#7 >= 50.00)) AND (ws_net_profit#7 <= 250.00))) - -(34) Project [codegen id : 8] -Output [5]: [ws_sold_date_sk#1, ws_quantity#5, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15] -Input [9]: [ws_sold_date_sk#1, ws_quantity#5, ws_net_profit#7, wr_refunded_addr_sk#10, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15, ca_address_sk#27, ca_state#28] - -(35) Scan parquet default.date_dim -Output [2]: [d_date_sk#31, d_year#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#31, d_year#32] - -(37) Filter [codegen id : 6] -Input [2]: [d_date_sk#31, d_year#32] -Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2000)) AND isnotnull(d_date_sk#31)) - -(38) Project [codegen id : 6] -Output [1]: [d_date_sk#31] -Input [2]: [d_date_sk#31, d_year#32] - -(39) BroadcastExchange -Input [1]: [d_date_sk#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] - -(40) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#31] -Join condition: None - -(41) Project [codegen id : 8] -Output [4]: [ws_quantity#5, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15] -Input [6]: [ws_sold_date_sk#1, ws_quantity#5, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15, d_date_sk#31] - -(42) Scan parquet default.reason -Output [2]: [r_reason_sk#34, r_reason_desc#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 7] -Input [2]: [r_reason_sk#34, r_reason_desc#35] - -(44) Filter [codegen id : 7] -Input [2]: [r_reason_sk#34, r_reason_desc#35] -Condition : isnotnull(r_reason_sk#34) - -(45) BroadcastExchange -Input [2]: [r_reason_sk#34, r_reason_desc#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] - -(46) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_reason_sk#12] -Right keys [1]: [cast(r_reason_sk#34 as bigint)] -Join condition: None - -(47) Project [codegen id : 8] -Output [4]: [ws_quantity#5, wr_fee#14, wr_refunded_cash#15, r_reason_desc#35] -Input [6]: [ws_quantity#5, wr_reason_sk#12, wr_fee#14, wr_refunded_cash#15, r_reason_sk#34, r_reason_desc#35] - -(48) HashAggregate [codegen id : 8] -Input [4]: [ws_quantity#5, wr_fee#14, wr_refunded_cash#15, r_reason_desc#35] -Keys [1]: [r_reason_desc#35] -Functions [3]: [partial_avg(cast(ws_quantity#5 as bigint)), partial_avg(UnscaledValue(wr_refunded_cash#15)), partial_avg(UnscaledValue(wr_fee#14))] -Aggregate Attributes [6]: [sum#37, count#38, sum#39, count#40, sum#41, count#42] -Results [7]: [r_reason_desc#35, sum#43, count#44, sum#45, count#46, sum#47, count#48] - -(49) Exchange -Input [7]: [r_reason_desc#35, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Arguments: hashpartitioning(r_reason_desc#35, 5), ENSURE_REQUIREMENTS, [id=#49] - -(50) HashAggregate [codegen id : 9] -Input [7]: [r_reason_desc#35, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Keys [1]: [r_reason_desc#35] -Functions [3]: [avg(cast(ws_quantity#5 as bigint)), avg(UnscaledValue(wr_refunded_cash#15)), avg(UnscaledValue(wr_fee#14))] -Aggregate Attributes [3]: [avg(cast(ws_quantity#5 as bigint))#50, avg(UnscaledValue(wr_refunded_cash#15))#51, avg(UnscaledValue(wr_fee#14))#52] -Results [4]: [substr(r_reason_desc#35, 1, 20) AS substr(r_reason_desc, 1, 20)#53, avg(cast(ws_quantity#5 as bigint))#50 AS avg(ws_quantity)#54, cast((avg(UnscaledValue(wr_refunded_cash#15))#51 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#55, cast((avg(UnscaledValue(wr_fee#14))#52 / 100.0) as decimal(11,6)) AS avg(wr_fee)#56] - -(51) TakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#53, avg(ws_quantity)#54, avg(wr_refunded_cash)#55, avg(wr_fee)#56] -Arguments: 100, [substr(r_reason_desc, 1, 20)#53 ASC NULLS FIRST, avg(ws_quantity)#54 ASC NULLS FIRST, avg(wr_refunded_cash)#55 ASC NULLS FIRST, avg(wr_fee)#56 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#53, avg(ws_quantity)#54, avg(wr_refunded_cash)#55, avg(wr_fee)#56] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q85/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q85/simplified.txt deleted file mode 100644 index b95873716..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q85/simplified.txt +++ /dev/null @@ -1,76 +0,0 @@ -TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (9) - HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(cast(ws_quantity as bigint)),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] - InputAdapter - Exchange [r_reason_desc] #1 - WholeStageCodegen (8) - HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - BroadcastHashJoin [wr_reason_sk,r_reason_sk] - Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] - Project [ws_sold_date_sk,ws_quantity,ws_net_profit,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] - Project [ws_sold_date_sk,ws_quantity,ws_net_profit,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] - Project [ws_sold_date_sk,ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_sold_date_sk,ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - Filter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sold_date_sk,ws_sales_price,ws_net_profit] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [wp_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [cd_demo_sk,cd_marital_status,cd_education_status] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [cd_demo_sk,cd_marital_status,cd_education_status] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [ca_address_sk,ca_state] - Filter [ca_country,ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - Filter [r_reason_sk] - ColumnarToRow - InputAdapter - Scan parquet default.reason [r_reason_sk,r_reason_desc] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q86/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q86/explain.txt deleted file mode 100644 index 20ae4d244..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q86/explain.txt +++ /dev/null @@ -1,142 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (25) -+- * Project (24) - +- Window (23) - +- * Sort (22) - +- Exchange (21) - +- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Expand (17) - +- * Project (16) - +- * BroadcastHashJoin Inner BuildRight (15) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.web_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- BroadcastExchange (14) - +- * Filter (13) - +- * ColumnarToRow (12) - +- Scan parquet default.item (11) - - -(1) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_net_paid#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_net_paid#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_net_paid#3] -Condition : (isnotnull(ws_sold_date_sk#1) AND isnotnull(ws_item_sk#2)) - -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_month_seq#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] -Condition : (((isnotnull(d_month_seq#5) AND (d_month_seq#5 >= 1200)) AND (d_month_seq#5 <= 1211)) AND isnotnull(d_date_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(10) Project [codegen id : 3] -Output [2]: [ws_item_sk#2, ws_net_paid#3] -Input [4]: [ws_sold_date_sk#1, ws_item_sk#2, ws_net_paid#3, d_date_sk#4] - -(11) Scan parquet default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#7, i_class#8, i_category#9] - -(13) Filter [codegen id : 2] -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(14) BroadcastExchange -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#7] -Join condition: None - -(16) Project [codegen id : 3] -Output [3]: [ws_net_paid#3, i_category#9, i_class#8] -Input [5]: [ws_item_sk#2, ws_net_paid#3, i_item_sk#7, i_class#8, i_category#9] - -(17) Expand [codegen id : 3] -Input [3]: [ws_net_paid#3, i_category#9, i_class#8] -Arguments: [List(ws_net_paid#3, i_category#9, i_class#8, 0), List(ws_net_paid#3, i_category#9, null, 1), List(ws_net_paid#3, null, null, 3)], [ws_net_paid#3, i_category#11, i_class#12, spark_grouping_id#13] - -(18) HashAggregate [codegen id : 3] -Input [4]: [ws_net_paid#3, i_category#11, i_class#12, spark_grouping_id#13] -Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#3))] -Aggregate Attributes [1]: [sum#14] -Results [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] - -(19) Exchange -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -Arguments: hashpartitioning(i_category#11, i_class#12, spark_grouping_id#13, 5), true, [id=#16] - -(20) HashAggregate [codegen id : 4] -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] -Functions [1]: [sum(UnscaledValue(ws_net_paid#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#3))#17] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#3))#17,17,2) AS total_sum#18, i_category#11, i_class#12, (cast((shiftright(spark_grouping_id#13, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint)) AS lochierarchy#19, (cast((shiftright(spark_grouping_id#13, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint)) AS _w1#20, CASE WHEN (cast(cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint) as int) = 0) THEN i_category#11 END AS _w2#21, MakeDecimal(sum(UnscaledValue(ws_net_paid#3))#17,17,2) AS _w3#22] - -(21) Exchange -Input [7]: [total_sum#18, i_category#11, i_class#12, lochierarchy#19, _w1#20, _w2#21, _w3#22] -Arguments: hashpartitioning(_w1#20, _w2#21, 5), true, [id=#23] - -(22) Sort [codegen id : 5] -Input [7]: [total_sum#18, i_category#11, i_class#12, lochierarchy#19, _w1#20, _w2#21, _w3#22] -Arguments: [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w3#22 DESC NULLS LAST], false, 0 - -(23) Window -Input [7]: [total_sum#18, i_category#11, i_class#12, lochierarchy#19, _w1#20, _w2#21, _w3#22] -Arguments: [rank(_w3#22) windowspecdefinition(_w1#20, _w2#21, _w3#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#24], [_w1#20, _w2#21], [_w3#22 DESC NULLS LAST] - -(24) Project [codegen id : 6] -Output [5]: [total_sum#18, i_category#11, i_class#12, lochierarchy#19, rank_within_parent#24] -Input [8]: [total_sum#18, i_category#11, i_class#12, lochierarchy#19, _w1#20, _w2#21, _w3#22, rank_within_parent#24] - -(25) TakeOrderedAndProject -Input [5]: [total_sum#18, i_category#11, i_class#12, lochierarchy#19, rank_within_parent#24] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (cast(lochierarchy#19 as int) = 0) THEN i_category#11 END ASC NULLS FIRST, rank_within_parent#24 ASC NULLS FIRST], [total_sum#18, i_category#11, i_class#12, lochierarchy#19, rank_within_parent#24] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q86/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q86/simplified.txt deleted file mode 100644 index cac22cb0f..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q86/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (6) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w3,_w1,_w2] - WholeStageCodegen (5) - Sort [_w1,_w2,_w3] - InputAdapter - Exchange [_w1,_w2] #1 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w1,_w2,_w3,sum] - InputAdapter - Exchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_net_paid] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_class,i_category] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q87/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q87/explain.txt deleted file mode 100644 index 3d59a670b..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q87/explain.txt +++ /dev/null @@ -1,323 +0,0 @@ -== Physical Plan == -* HashAggregate (54) -+- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * BroadcastHashJoin LeftAnti BuildRight (44) - :- * BroadcastHashJoin LeftAnti BuildRight (30) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer (11) - : +- BroadcastExchange (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) - - -(1) Scan parquet default.store_sales -Output [2]: [ss_sold_date_sk#1, ss_customer_sk#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 11] -Input [2]: [ss_sold_date_sk#1, ss_customer_sk#2] - -(3) Filter [codegen id : 11] -Input [2]: [ss_sold_date_sk#1, ss_customer_sk#2] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_customer_sk#2)) - -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#3, d_date#4, d_month_seq#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#3, d_date#4, d_month_seq#5] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#3, d_date#4, d_month_seq#5] -Condition : (((isnotnull(d_month_seq#5) AND (d_month_seq#5 >= 1200)) AND (d_month_seq#5 <= 1211)) AND isnotnull(d_date_sk#3)) - -(7) Project [codegen id : 1] -Output [2]: [d_date_sk#3, d_date#4] -Input [3]: [d_date_sk#3, d_date#4, d_month_seq#5] - -(8) BroadcastExchange -Input [2]: [d_date_sk#3, d_date#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] - -(9) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#3] -Join condition: None - -(10) Project [codegen id : 11] -Output [2]: [ss_customer_sk#2, d_date#4] -Input [4]: [ss_sold_date_sk#1, ss_customer_sk#2, d_date_sk#3, d_date#4] - -(11) Scan parquet default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] - -(13) Filter [codegen id : 2] -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(14) BroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(15) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#7] -Join condition: None - -(16) Project [codegen id : 11] -Output [3]: [d_date#4, c_first_name#8, c_last_name#9] -Input [5]: [ss_customer_sk#2, d_date#4, c_customer_sk#7, c_first_name#8, c_last_name#9] - -(17) Scan parquet default.catalog_sales -Output [2]: [cs_sold_date_sk#11, cs_bill_customer_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 5] -Input [2]: [cs_sold_date_sk#11, cs_bill_customer_sk#12] - -(19) Filter [codegen id : 5] -Input [2]: [cs_sold_date_sk#11, cs_bill_customer_sk#12] -Condition : (isnotnull(cs_sold_date_sk#11) AND isnotnull(cs_bill_customer_sk#12)) - -(20) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#13, d_date#14] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(22) Project [codegen id : 5] -Output [2]: [cs_bill_customer_sk#12, d_date#14] -Input [4]: [cs_sold_date_sk#11, cs_bill_customer_sk#12, d_date_sk#13, d_date#14] - -(23) ReusedExchange [Reuses operator id: 14] -Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_bill_customer_sk#12] -Right keys [1]: [c_customer_sk#15] -Join condition: None - -(25) Project [codegen id : 5] -Output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Input [5]: [cs_bill_customer_sk#12, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] - -(26) HashAggregate [codegen id : 5] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(27) Exchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), true, [id=#18] - -(28) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(29) BroadcastExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -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=#19] - -(30) BroadcastHashJoin [codegen id : 11] -Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#4, 0), isnull(d_date#4)] -Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 0), isnull(d_date#14)] -Join condition: None - -(31) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#20, ws_bill_customer_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) ColumnarToRow [codegen id : 9] -Input [2]: [ws_sold_date_sk#20, ws_bill_customer_sk#21] - -(33) Filter [codegen id : 9] -Input [2]: [ws_sold_date_sk#20, ws_bill_customer_sk#21] -Condition : (isnotnull(ws_sold_date_sk#20) AND isnotnull(ws_bill_customer_sk#21)) - -(34) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#22, d_date#23] - -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join condition: None - -(36) Project [codegen id : 9] -Output [2]: [ws_bill_customer_sk#21, d_date#23] -Input [4]: [ws_sold_date_sk#20, ws_bill_customer_sk#21, d_date_sk#22, d_date#23] - -(37) ReusedExchange [Reuses operator id: 14] -Output [3]: [c_customer_sk#24, c_first_name#25, c_last_name#26] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_bill_customer_sk#21] -Right keys [1]: [c_customer_sk#24] -Join condition: None - -(39) Project [codegen id : 9] -Output [3]: [c_last_name#26, c_first_name#25, d_date#23] -Input [5]: [ws_bill_customer_sk#21, d_date#23, c_customer_sk#24, c_first_name#25, c_last_name#26] - -(40) HashAggregate [codegen id : 9] -Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -Keys [3]: [c_last_name#26, c_first_name#25, d_date#23] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#26, c_first_name#25, d_date#23] - -(41) Exchange -Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -Arguments: hashpartitioning(c_last_name#26, c_first_name#25, d_date#23, 5), true, [id=#27] - -(42) HashAggregate [codegen id : 10] -Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -Keys [3]: [c_last_name#26, c_first_name#25, d_date#23] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#26, c_first_name#25, d_date#23] - -(43) BroadcastExchange -Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -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=#28] - -(44) BroadcastHashJoin [codegen id : 11] -Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#4, 0), isnull(d_date#4)] -Right keys [6]: [coalesce(c_last_name#26, ), isnull(c_last_name#26), coalesce(c_first_name#25, ), isnull(c_first_name#25), coalesce(d_date#23, 0), isnull(d_date#23)] -Join condition: None - -(45) HashAggregate [codegen id : 11] -Input [3]: [d_date#4, c_first_name#8, c_last_name#9] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(46) Exchange -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#4, 5), true, [id=#29] - -(47) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(48) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(49) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(50) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(51) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results: [] - -(52) HashAggregate [codegen id : 12] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [1]: [count#31] - -(53) Exchange -Input [1]: [count#31] -Arguments: SinglePartition, true, [id=#32] - -(54) HashAggregate [codegen id : 13] -Input [1]: [count#31] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [1]: [count(1)#33 AS count(1)#34] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q87/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q87/simplified.txt deleted file mode 100644 index a5b57a4ac..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q87/simplified.txt +++ /dev/null @@ -1,80 +0,0 @@ -WholeStageCodegen (13) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (12) - HashAggregate [count,count] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (11) - HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - Project [d_date,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q88/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q88/explain.txt deleted file mode 100644 index 22297e02e..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q88/explain.txt +++ /dev/null @@ -1,960 +0,0 @@ -== Physical Plan == -BroadcastNestedLoopJoin Inner BuildRight (174) -:- BroadcastNestedLoopJoin Inner BuildRight (153) -: :- BroadcastNestedLoopJoin Inner BuildRight (132) -: : :- BroadcastNestedLoopJoin Inner BuildRight (111) -: : : :- BroadcastNestedLoopJoin Inner BuildRight (90) -: : : : :- BroadcastNestedLoopJoin Inner BuildRight (69) -: : : : : :- BroadcastNestedLoopJoin Inner BuildRight (48) -: : : : : : :- * HashAggregate (27) -: : : : : : : +- Exchange (26) -: : : : : : : +- * HashAggregate (25) -: : : : : : : +- * Project (24) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) -: : : : : : : :- * Project (17) -: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) -: : : : : : : : :- * Project (10) -: : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) -: : : : : : : : : :- * Filter (3) -: : : : : : : : : : +- * ColumnarToRow (2) -: : : : : : : : : : +- Scan parquet default.store_sales (1) -: : : : : : : : : +- BroadcastExchange (8) -: : : : : : : : : +- * Project (7) -: : : : : : : : : +- * Filter (6) -: : : : : : : : : +- * ColumnarToRow (5) -: : : : : : : : : +- Scan parquet default.household_demographics (4) -: : : : : : : : +- BroadcastExchange (15) -: : : : : : : : +- * Project (14) -: : : : : : : : +- * Filter (13) -: : : : : : : : +- * ColumnarToRow (12) -: : : : : : : : +- Scan parquet default.time_dim (11) -: : : : : : : +- BroadcastExchange (22) -: : : : : : : +- * Project (21) -: : : : : : : +- * Filter (20) -: : : : : : : +- * ColumnarToRow (19) -: : : : : : : +- Scan parquet default.store (18) -: : : : : : +- BroadcastExchange (47) -: : : : : : +- * HashAggregate (46) -: : : : : : +- Exchange (45) -: : : : : : +- * HashAggregate (44) -: : : : : : +- * Project (43) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (42) -: : : : : : :- * Project (40) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) -: : : : : : : :- * Project (33) -: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (32) -: : : : : : : : :- * Filter (30) -: : : : : : : : : +- * ColumnarToRow (29) -: : : : : : : : : +- Scan parquet default.store_sales (28) -: : : : : : : : +- ReusedExchange (31) -: : : : : : : +- BroadcastExchange (38) -: : : : : : : +- * Project (37) -: : : : : : : +- * Filter (36) -: : : : : : : +- * ColumnarToRow (35) -: : : : : : : +- Scan parquet default.time_dim (34) -: : : : : : +- ReusedExchange (41) -: : : : : +- BroadcastExchange (68) -: : : : : +- * HashAggregate (67) -: : : : : +- Exchange (66) -: : : : : +- * HashAggregate (65) -: : : : : +- * Project (64) -: : : : : +- * BroadcastHashJoin Inner BuildRight (63) -: : : : : :- * Project (61) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (60) -: : : : : : :- * Project (54) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) -: : : : : : : :- * Filter (51) -: : : : : : : : +- * ColumnarToRow (50) -: : : : : : : : +- Scan parquet default.store_sales (49) -: : : : : : : +- ReusedExchange (52) -: : : : : : +- BroadcastExchange (59) -: : : : : : +- * Project (58) -: : : : : : +- * Filter (57) -: : : : : : +- * ColumnarToRow (56) -: : : : : : +- Scan parquet default.time_dim (55) -: : : : : +- ReusedExchange (62) -: : : : +- BroadcastExchange (89) -: : : : +- * HashAggregate (88) -: : : : +- Exchange (87) -: : : : +- * HashAggregate (86) -: : : : +- * Project (85) -: : : : +- * BroadcastHashJoin Inner BuildRight (84) -: : : : :- * Project (82) -: : : : : +- * BroadcastHashJoin Inner BuildRight (81) -: : : : : :- * Project (75) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (74) -: : : : : : :- * Filter (72) -: : : : : : : +- * ColumnarToRow (71) -: : : : : : : +- Scan parquet default.store_sales (70) -: : : : : : +- ReusedExchange (73) -: : : : : +- BroadcastExchange (80) -: : : : : +- * Project (79) -: : : : : +- * Filter (78) -: : : : : +- * ColumnarToRow (77) -: : : : : +- Scan parquet default.time_dim (76) -: : : : +- ReusedExchange (83) -: : : +- BroadcastExchange (110) -: : : +- * HashAggregate (109) -: : : +- Exchange (108) -: : : +- * HashAggregate (107) -: : : +- * Project (106) -: : : +- * BroadcastHashJoin Inner BuildRight (105) -: : : :- * Project (103) -: : : : +- * BroadcastHashJoin Inner BuildRight (102) -: : : : :- * Project (96) -: : : : : +- * BroadcastHashJoin Inner BuildRight (95) -: : : : : :- * Filter (93) -: : : : : : +- * ColumnarToRow (92) -: : : : : : +- Scan parquet default.store_sales (91) -: : : : : +- ReusedExchange (94) -: : : : +- BroadcastExchange (101) -: : : : +- * Project (100) -: : : : +- * Filter (99) -: : : : +- * ColumnarToRow (98) -: : : : +- Scan parquet default.time_dim (97) -: : : +- ReusedExchange (104) -: : +- BroadcastExchange (131) -: : +- * HashAggregate (130) -: : +- Exchange (129) -: : +- * HashAggregate (128) -: : +- * Project (127) -: : +- * BroadcastHashJoin Inner BuildRight (126) -: : :- * Project (124) -: : : +- * BroadcastHashJoin Inner BuildRight (123) -: : : :- * Project (117) -: : : : +- * BroadcastHashJoin Inner BuildRight (116) -: : : : :- * Filter (114) -: : : : : +- * ColumnarToRow (113) -: : : : : +- Scan parquet default.store_sales (112) -: : : : +- ReusedExchange (115) -: : : +- BroadcastExchange (122) -: : : +- * Project (121) -: : : +- * Filter (120) -: : : +- * ColumnarToRow (119) -: : : +- Scan parquet default.time_dim (118) -: : +- ReusedExchange (125) -: +- BroadcastExchange (152) -: +- * HashAggregate (151) -: +- Exchange (150) -: +- * HashAggregate (149) -: +- * Project (148) -: +- * BroadcastHashJoin Inner BuildRight (147) -: :- * Project (145) -: : +- * BroadcastHashJoin Inner BuildRight (144) -: : :- * Project (138) -: : : +- * BroadcastHashJoin Inner BuildRight (137) -: : : :- * Filter (135) -: : : : +- * ColumnarToRow (134) -: : : : +- Scan parquet default.store_sales (133) -: : : +- ReusedExchange (136) -: : +- BroadcastExchange (143) -: : +- * Project (142) -: : +- * Filter (141) -: : +- * ColumnarToRow (140) -: : +- Scan parquet default.time_dim (139) -: +- ReusedExchange (146) -+- BroadcastExchange (173) - +- * HashAggregate (172) - +- Exchange (171) - +- * HashAggregate (170) - +- * Project (169) - +- * BroadcastHashJoin Inner BuildRight (168) - :- * Project (166) - : +- * BroadcastHashJoin Inner BuildRight (165) - : :- * Project (159) - : : +- * BroadcastHashJoin Inner BuildRight (158) - : : :- * Filter (156) - : : : +- * ColumnarToRow (155) - : : : +- Scan parquet default.store_sales (154) - : : +- ReusedExchange (157) - : +- BroadcastExchange (164) - : +- * Project (163) - : +- * Filter (162) - : +- * ColumnarToRow (161) - : +- Scan parquet default.time_dim (160) - +- ReusedExchange (167) - - -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet default.household_demographics -Output [3]: [hd_demo_sk#4, hd_dep_count#5, hd_vehicle_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [hd_demo_sk#4, hd_dep_count#5, hd_vehicle_count#6] - -(6) Filter [codegen id : 1] -Input [3]: [hd_demo_sk#4, hd_dep_count#5, hd_vehicle_count#6] -Condition : (((((hd_dep_count#5 = 4) AND (hd_vehicle_count#6 <= 6)) OR ((hd_dep_count#5 = 2) AND (hd_vehicle_count#6 <= 4))) OR ((hd_dep_count#5 = 0) AND (hd_vehicle_count#6 <= 2))) AND isnotnull(hd_demo_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [hd_demo_sk#4] -Input [3]: [hd_demo_sk#4, hd_dep_count#5, hd_vehicle_count#6] - -(8) BroadcastExchange -Input [1]: [hd_demo_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(10) Project [codegen id : 4] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#4] - -(11) Scan parquet default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(13) Filter [codegen id : 2] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(14) Project [codegen id : 2] -Output [1]: [t_time_sk#8] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(15) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] -Join condition: None - -(17) Project [codegen id : 4] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] - -(18) Scan parquet default.store -Output [2]: [s_store_sk#12, s_store_name#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#12, s_store_name#13] - -(20) Filter [codegen id : 3] -Input [2]: [s_store_sk#12, s_store_name#13] -Condition : ((isnotnull(s_store_name#13) AND (s_store_name#13 = ese)) AND isnotnull(s_store_sk#12)) - -(21) Project [codegen id : 3] -Output [1]: [s_store_sk#12] -Input [2]: [s_store_sk#12, s_store_name#13] - -(22) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] -Join condition: None - -(24) Project [codegen id : 4] -Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#12] - -(25) HashAggregate [codegen id : 4] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [1]: [count#16] - -(26) Exchange -Input [1]: [count#16] -Arguments: SinglePartition, true, [id=#17] - -(27) HashAggregate [codegen id : 5] -Input [1]: [count#16] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#18] -Results [1]: [count(1)#18 AS h8_30_to_9#19] - -(28) Scan parquet default.store_sales -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 9] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(30) Filter [codegen id : 9] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(31) ReusedExchange [Reuses operator id: 8] -Output [1]: [hd_demo_sk#4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(33) Project [codegen id : 9] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#4] - -(34) Scan parquet default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 7] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(36) Filter [codegen id : 7] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 9)) AND (t_minute#10 < 30)) AND isnotnull(t_time_sk#8)) - -(37) Project [codegen id : 7] -Output [1]: [t_time_sk#8] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(38) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] -Join condition: None - -(40) Project [codegen id : 9] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] - -(41) ReusedExchange [Reuses operator id: 22] -Output [1]: [s_store_sk#12] - -(42) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] -Join condition: None - -(43) Project [codegen id : 9] -Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#12] - -(44) HashAggregate [codegen id : 9] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#21] -Results [1]: [count#22] - -(45) Exchange -Input [1]: [count#22] -Arguments: SinglePartition, true, [id=#23] - -(46) HashAggregate [codegen id : 10] -Input [1]: [count#22] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#24] -Results [1]: [count(1)#24 AS h9_to_9_30#25] - -(47) BroadcastExchange -Input [1]: [h9_to_9_30#25] -Arguments: IdentityBroadcastMode, [id=#26] - -(48) BroadcastNestedLoopJoin -Join condition: None - -(49) Scan parquet default.store_sales -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(50) ColumnarToRow [codegen id : 14] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(51) Filter [codegen id : 14] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(52) ReusedExchange [Reuses operator id: 8] -Output [1]: [hd_demo_sk#4] - -(53) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(54) Project [codegen id : 14] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#4] - -(55) Scan parquet default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(56) ColumnarToRow [codegen id : 12] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(57) Filter [codegen id : 12] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 9)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(58) Project [codegen id : 12] -Output [1]: [t_time_sk#8] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(59) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] - -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] -Join condition: None - -(61) Project [codegen id : 14] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] - -(62) ReusedExchange [Reuses operator id: 22] -Output [1]: [s_store_sk#12] - -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] -Join condition: None - -(64) Project [codegen id : 14] -Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#12] - -(65) HashAggregate [codegen id : 14] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] - -(66) Exchange -Input [1]: [count#29] -Arguments: SinglePartition, true, [id=#30] - -(67) HashAggregate [codegen id : 15] -Input [1]: [count#29] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [1]: [count(1)#31 AS h9_30_to_10#32] - -(68) BroadcastExchange -Input [1]: [h9_30_to_10#32] -Arguments: IdentityBroadcastMode, [id=#33] - -(69) BroadcastNestedLoopJoin -Join condition: None - -(70) Scan parquet default.store_sales -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 19] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(72) Filter [codegen id : 19] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(73) ReusedExchange [Reuses operator id: 8] -Output [1]: [hd_demo_sk#4] - -(74) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(75) Project [codegen id : 19] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#4] - -(76) Scan parquet default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(77) ColumnarToRow [codegen id : 17] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(78) Filter [codegen id : 17] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 10)) AND (t_minute#10 < 30)) AND isnotnull(t_time_sk#8)) - -(79) Project [codegen id : 17] -Output [1]: [t_time_sk#8] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(80) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] - -(81) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] -Join condition: None - -(82) Project [codegen id : 19] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] - -(83) ReusedExchange [Reuses operator id: 22] -Output [1]: [s_store_sk#12] - -(84) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] -Join condition: None - -(85) Project [codegen id : 19] -Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#12] - -(86) HashAggregate [codegen id : 19] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [1]: [count#36] - -(87) Exchange -Input [1]: [count#36] -Arguments: SinglePartition, true, [id=#37] - -(88) HashAggregate [codegen id : 20] -Input [1]: [count#36] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [1]: [count(1)#38 AS h10_to_10_30#39] - -(89) BroadcastExchange -Input [1]: [h10_to_10_30#39] -Arguments: IdentityBroadcastMode, [id=#40] - -(90) BroadcastNestedLoopJoin -Join condition: None - -(91) Scan parquet default.store_sales -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(92) ColumnarToRow [codegen id : 24] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(93) Filter [codegen id : 24] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(94) ReusedExchange [Reuses operator id: 8] -Output [1]: [hd_demo_sk#4] - -(95) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(96) Project [codegen id : 24] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#4] - -(97) Scan parquet default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(98) ColumnarToRow [codegen id : 22] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(99) Filter [codegen id : 22] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 10)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(100) Project [codegen id : 22] -Output [1]: [t_time_sk#8] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(101) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] - -(102) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] -Join condition: None - -(103) Project [codegen id : 24] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] - -(104) ReusedExchange [Reuses operator id: 22] -Output [1]: [s_store_sk#12] - -(105) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] -Join condition: None - -(106) Project [codegen id : 24] -Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#12] - -(107) HashAggregate [codegen id : 24] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#42] -Results [1]: [count#43] - -(108) Exchange -Input [1]: [count#43] -Arguments: SinglePartition, true, [id=#44] - -(109) HashAggregate [codegen id : 25] -Input [1]: [count#43] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#45] -Results [1]: [count(1)#45 AS h10_30_to_11#46] - -(110) BroadcastExchange -Input [1]: [h10_30_to_11#46] -Arguments: IdentityBroadcastMode, [id=#47] - -(111) BroadcastNestedLoopJoin -Join condition: None - -(112) Scan parquet default.store_sales -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(113) ColumnarToRow [codegen id : 29] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(114) Filter [codegen id : 29] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(115) ReusedExchange [Reuses operator id: 8] -Output [1]: [hd_demo_sk#4] - -(116) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(117) Project [codegen id : 29] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#4] - -(118) Scan parquet default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(119) ColumnarToRow [codegen id : 27] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(120) Filter [codegen id : 27] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 11)) AND (t_minute#10 < 30)) AND isnotnull(t_time_sk#8)) - -(121) Project [codegen id : 27] -Output [1]: [t_time_sk#8] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(122) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] - -(123) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] -Join condition: None - -(124) Project [codegen id : 29] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] - -(125) ReusedExchange [Reuses operator id: 22] -Output [1]: [s_store_sk#12] - -(126) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] -Join condition: None - -(127) Project [codegen id : 29] -Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#12] - -(128) HashAggregate [codegen id : 29] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#49] -Results [1]: [count#50] - -(129) Exchange -Input [1]: [count#50] -Arguments: SinglePartition, true, [id=#51] - -(130) HashAggregate [codegen id : 30] -Input [1]: [count#50] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#52] -Results [1]: [count(1)#52 AS h11_to_11_30#53] - -(131) BroadcastExchange -Input [1]: [h11_to_11_30#53] -Arguments: IdentityBroadcastMode, [id=#54] - -(132) BroadcastNestedLoopJoin -Join condition: None - -(133) Scan parquet default.store_sales -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(134) ColumnarToRow [codegen id : 34] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(135) Filter [codegen id : 34] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(136) ReusedExchange [Reuses operator id: 8] -Output [1]: [hd_demo_sk#4] - -(137) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(138) Project [codegen id : 34] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#4] - -(139) Scan parquet default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(140) ColumnarToRow [codegen id : 32] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(141) Filter [codegen id : 32] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 11)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(142) Project [codegen id : 32] -Output [1]: [t_time_sk#8] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(143) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#55] - -(144) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] -Join condition: None - -(145) Project [codegen id : 34] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] - -(146) ReusedExchange [Reuses operator id: 22] -Output [1]: [s_store_sk#12] - -(147) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] -Join condition: None - -(148) Project [codegen id : 34] -Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#12] - -(149) HashAggregate [codegen id : 34] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#56] -Results [1]: [count#57] - -(150) Exchange -Input [1]: [count#57] -Arguments: SinglePartition, true, [id=#58] - -(151) HashAggregate [codegen id : 35] -Input [1]: [count#57] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#59] -Results [1]: [count(1)#59 AS h11_30_to_12#60] - -(152) BroadcastExchange -Input [1]: [h11_30_to_12#60] -Arguments: IdentityBroadcastMode, [id=#61] - -(153) BroadcastNestedLoopJoin -Join condition: None - -(154) Scan parquet default.store_sales -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(155) ColumnarToRow [codegen id : 39] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(156) Filter [codegen id : 39] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(157) ReusedExchange [Reuses operator id: 8] -Output [1]: [hd_demo_sk#4] - -(158) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(159) Project [codegen id : 39] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#4] - -(160) Scan parquet default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(161) ColumnarToRow [codegen id : 37] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(162) Filter [codegen id : 37] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 12)) AND (t_minute#10 < 30)) AND isnotnull(t_time_sk#8)) - -(163) Project [codegen id : 37] -Output [1]: [t_time_sk#8] -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] - -(164) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#62] - -(165) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] -Join condition: None - -(166) Project [codegen id : 39] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] - -(167) ReusedExchange [Reuses operator id: 22] -Output [1]: [s_store_sk#12] - -(168) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] -Join condition: None - -(169) Project [codegen id : 39] -Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#12] - -(170) HashAggregate [codegen id : 39] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#63] -Results [1]: [count#64] - -(171) Exchange -Input [1]: [count#64] -Arguments: SinglePartition, true, [id=#65] - -(172) HashAggregate [codegen id : 40] -Input [1]: [count#64] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#66] -Results [1]: [count(1)#66 AS h12_to_12_30#67] - -(173) BroadcastExchange -Input [1]: [h12_to_12_30#67] -Arguments: IdentityBroadcastMode, [id=#68] - -(174) BroadcastNestedLoopJoin -Join condition: None - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q88/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q88/simplified.txt deleted file mode 100644 index 8e72594b0..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q88/simplified.txt +++ /dev/null @@ -1,250 +0,0 @@ -BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - WholeStageCodegen (5) - HashAggregate [count] [count(1),h8_30_to_9,count] - InputAdapter - Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [t_time_sk] - Filter [t_hour,t_minute,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [s_store_sk] - Filter [s_store_name,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name] - BroadcastExchange #5 - WholeStageCodegen (10) - HashAggregate [count] [count(1),h9_to_9_30,count] - InputAdapter - Exchange #6 - WholeStageCodegen (9) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Project [t_time_sk] - Filter [t_hour,t_minute,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 - BroadcastExchange #8 - WholeStageCodegen (15) - HashAggregate [count] [count(1),h9_30_to_10,count] - InputAdapter - Exchange #9 - WholeStageCodegen (14) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (12) - Project [t_time_sk] - Filter [t_hour,t_minute,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 - BroadcastExchange #11 - WholeStageCodegen (20) - HashAggregate [count] [count(1),h10_to_10_30,count] - InputAdapter - Exchange #12 - WholeStageCodegen (19) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (17) - Project [t_time_sk] - Filter [t_hour,t_minute,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 - BroadcastExchange #14 - WholeStageCodegen (25) - HashAggregate [count] [count(1),h10_30_to_11,count] - InputAdapter - Exchange #15 - WholeStageCodegen (24) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (22) - Project [t_time_sk] - Filter [t_hour,t_minute,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 - BroadcastExchange #17 - WholeStageCodegen (30) - HashAggregate [count] [count(1),h11_to_11_30,count] - InputAdapter - Exchange #18 - WholeStageCodegen (29) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (27) - Project [t_time_sk] - Filter [t_hour,t_minute,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 - BroadcastExchange #20 - WholeStageCodegen (35) - HashAggregate [count] [count(1),h11_30_to_12,count] - InputAdapter - Exchange #21 - WholeStageCodegen (34) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #22 - WholeStageCodegen (32) - Project [t_time_sk] - Filter [t_hour,t_minute,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 - BroadcastExchange #23 - WholeStageCodegen (40) - HashAggregate [count] [count(1),h12_to_12_30,count] - InputAdapter - Exchange #24 - WholeStageCodegen (39) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #25 - WholeStageCodegen (37) - Project [t_time_sk] - Filter [t_hour,t_minute,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q89/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q89/explain.txt deleted file mode 100644 index 8dca84461..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q89/explain.txt +++ /dev/null @@ -1,175 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (31) -+- * Project (30) - +- * Filter (29) - +- Window (28) - +- * Sort (27) - +- Exchange (26) - +- * HashAggregate (25) - +- Exchange (24) - +- * HashAggregate (23) - +- * Project (22) - +- * BroadcastHashJoin Inner BuildRight (21) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.item (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.store_sales (4) - : +- BroadcastExchange (14) - : +- * Project (13) - : +- * Filter (12) - : +- * ColumnarToRow (11) - : +- Scan parquet default.date_dim (10) - +- BroadcastExchange (20) - +- * Filter (19) - +- * ColumnarToRow (18) - +- Scan parquet default.store (17) - - -(1) Scan parquet default.item -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)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -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)) - -(4) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [ss_sold_date_sk#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8] - -(6) Filter [codegen id : 1] -Input [4]: [ss_sold_date_sk#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8] -Condition : ((isnotnull(ss_item_sk#6) AND isnotnull(ss_sold_date_sk#5)) AND isnotnull(ss_store_sk#7)) - -(7) BroadcastExchange -Input [4]: [ss_sold_date_sk#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#9] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#6] -Join condition: None - -(9) Project [codegen id : 4] -Output [6]: [i_brand#2, i_class#3, i_category#4, ss_sold_date_sk#5, ss_store_sk#7, ss_sales_price#8] -Input [8]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, ss_sold_date_sk#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8] - -(10) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_moy#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] - -(12) Filter [codegen id : 2] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Condition : ((isnotnull(d_year#11) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) - -(13) Project [codegen id : 2] -Output [2]: [d_date_sk#10, d_moy#12] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] - -(14) BroadcastExchange -Input [2]: [d_date_sk#10, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(16) Project [codegen id : 4] -Output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#7, ss_sales_price#8, d_moy#12] -Input [8]: [i_brand#2, i_class#3, i_category#4, ss_sold_date_sk#5, ss_store_sk#7, ss_sales_price#8, d_date_sk#10, d_moy#12] - -(17) Scan parquet default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(19) Filter [codegen id : 3] -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : isnotnull(s_store_sk#14) - -(20) BroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#14] -Join condition: None - -(22) Project [codegen id : 4] -Output [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#8, d_moy#12, s_store_name#15, s_company_name#16] -Input [9]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#7, ss_sales_price#8, d_moy#12, s_store_sk#14, s_store_name#15, s_company_name#16] - -(23) HashAggregate [codegen id : 4] -Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#8, d_moy#12, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum#18] -Results [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum#19] - -(24) Exchange -Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum#19] -Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, 5), true, [id=#20] - -(25) HashAggregate [codegen id : 5] -Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum#19] -Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#21] -Results [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#21,17,2) AS sum_sales#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#21,17,2) AS _w0#23] - -(26) Exchange -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum_sales#22, _w0#23] -Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#15, s_company_name#16, 5), true, [id=#24] - -(27) Sort [codegen id : 6] -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum_sales#22, _w0#23] -Arguments: [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST], false, 0 - -(28) Window -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum_sales#22, _w0#23] -Arguments: [avg(_w0#23) windowspecdefinition(i_category#4, i_brand#2, s_store_name#15, s_company_name#16, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#25], [i_category#4, i_brand#2, s_store_name#15, s_company_name#16] - -(29) Filter [codegen id : 7] -Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum_sales#22, _w0#23, avg_monthly_sales#25] -Condition : (CASE WHEN NOT (avg_monthly_sales#25 = 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000) - -(30) Project [codegen id : 7] -Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum_sales#22, avg_monthly_sales#25] -Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum_sales#22, _w0#23, avg_monthly_sales#25] - -(31) TakeOrderedAndProject -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum_sales#22, avg_monthly_sales#25] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#15, s_company_name#16, d_moy#12, sum_sales#22, avg_monthly_sales#25] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q89/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q89/simplified.txt deleted file mode 100644 index efeab7f69..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q89/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (7) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - Sort [i_category,i_brand,s_store_name,s_company_name] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_sold_date_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_category,i_class,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_class,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date_sk,d_moy] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_company_name] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q9/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q9/explain.txt deleted file mode 100644 index 55f7977b2..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q9/explain.txt +++ /dev/null @@ -1,718 +0,0 @@ -== Physical Plan == -* Project (4) -+- * Filter (3) - +- * ColumnarToRow (2) - +- Scan parquet default.reason (1) - - -(1) Scan parquet default.reason -Output [1]: [r_reason_sk#1] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [1]: [r_reason_sk#1] - -(3) Filter [codegen id : 1] -Input [1]: [r_reason_sk#1] -Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) - -(4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3] > 62316685) THEN Subquery scalar-subquery#4, [id=#5] ELSE Subquery scalar-subquery#6, [id=#7] END AS bucket1#8, CASE WHEN (Subquery scalar-subquery#9, [id=#10] > 19045798) THEN Subquery scalar-subquery#11, [id=#12] ELSE Subquery scalar-subquery#13, [id=#14] END AS bucket2#15, CASE WHEN (Subquery scalar-subquery#16, [id=#17] > 365541424) THEN Subquery scalar-subquery#18, [id=#19] ELSE Subquery scalar-subquery#20, [id=#21] END AS bucket3#22, CASE WHEN (Subquery scalar-subquery#23, [id=#24] > 216357808) THEN Subquery scalar-subquery#25, [id=#26] ELSE Subquery scalar-subquery#27, [id=#28] END AS bucket4#29, CASE WHEN (Subquery scalar-subquery#30, [id=#31] > 184483884) THEN Subquery scalar-subquery#32, [id=#33] ELSE Subquery scalar-subquery#34, [id=#35] END AS bucket5#36] -Input [1]: [r_reason_sk#1] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* HashAggregate (11) -+- Exchange (10) - +- * HashAggregate (9) - +- * Project (8) - +- * Filter (7) - +- * ColumnarToRow (6) - +- Scan parquet default.store_sales (5) - - -(5) Scan parquet default.store_sales -Output [1]: [ss_quantity#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [1]: [ss_quantity#37] - -(7) Filter [codegen id : 1] -Input [1]: [ss_quantity#37] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) - -(8) Project [codegen id : 1] -Output: [] -Input [1]: [ss_quantity#37] - -(9) HashAggregate [codegen id : 1] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#38] -Results [1]: [count#39] - -(10) Exchange -Input [1]: [count#39] -Arguments: SinglePartition, true, [id=#40] - -(11) HashAggregate [codegen id : 2] -Input [1]: [count#39] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#41] -Results [1]: [count(1)#41 AS count(1)#42] - -Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#4, [id=#5] -* HashAggregate (18) -+- Exchange (17) - +- * HashAggregate (16) - +- * Project (15) - +- * Filter (14) - +- * ColumnarToRow (13) - +- Scan parquet default.store_sales (12) - - -(12) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_ext_discount_amt#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] - -(14) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) - -(15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#43] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] - -(16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#43] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#43))] -Aggregate Attributes [2]: [sum#44, count#45] -Results [2]: [sum#46, count#47] - -(17) Exchange -Input [2]: [sum#46, count#47] -Arguments: SinglePartition, true, [id=#48] - -(18) HashAggregate [codegen id : 2] -Input [2]: [sum#46, count#47] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#43))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#43))#49] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#43))#49 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#50] - -Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] -* HashAggregate (25) -+- Exchange (24) - +- * HashAggregate (23) - +- * Project (22) - +- * Filter (21) - +- * ColumnarToRow (20) - +- Scan parquet default.store_sales (19) - - -(19) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_net_paid#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_net_paid#51] - -(21) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_net_paid#51] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) - -(22) Project [codegen id : 1] -Output [1]: [ss_net_paid#51] -Input [2]: [ss_quantity#37, ss_net_paid#51] - -(23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#51] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [2]: [sum#52, count#53] -Results [2]: [sum#54, count#55] - -(24) Exchange -Input [2]: [sum#54, count#55] -Arguments: SinglePartition, true, [id=#56] - -(25) HashAggregate [codegen id : 2] -Input [2]: [sum#54, count#55] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#51))#57] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#51))#57 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#58] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* HashAggregate (32) -+- Exchange (31) - +- * HashAggregate (30) - +- * Project (29) - +- * Filter (28) - +- * ColumnarToRow (27) - +- Scan parquet default.store_sales (26) - - -(26) Scan parquet default.store_sales -Output [1]: [ss_quantity#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(27) ColumnarToRow [codegen id : 1] -Input [1]: [ss_quantity#37] - -(28) Filter [codegen id : 1] -Input [1]: [ss_quantity#37] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) - -(29) Project [codegen id : 1] -Output: [] -Input [1]: [ss_quantity#37] - -(30) HashAggregate [codegen id : 1] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#59] -Results [1]: [count#60] - -(31) Exchange -Input [1]: [count#60] -Arguments: SinglePartition, true, [id=#61] - -(32) HashAggregate [codegen id : 2] -Input [1]: [count#60] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#62] -Results [1]: [count(1)#62 AS count(1)#63] - -Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* HashAggregate (39) -+- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * Filter (35) - +- * ColumnarToRow (34) - +- Scan parquet default.store_sales (33) - - -(33) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_ext_discount_amt#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] - -(35) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) - -(36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#43] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] - -(37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#43] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#43))] -Aggregate Attributes [2]: [sum#64, count#65] -Results [2]: [sum#66, count#67] - -(38) Exchange -Input [2]: [sum#66, count#67] -Arguments: SinglePartition, true, [id=#68] - -(39) HashAggregate [codegen id : 2] -Input [2]: [sum#66, count#67] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#43))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#43))#69] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#43))#69 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#70] - -Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] -* HashAggregate (46) -+- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * Filter (42) - +- * ColumnarToRow (41) - +- Scan parquet default.store_sales (40) - - -(40) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_net_paid#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_net_paid#51] - -(42) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_net_paid#51] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) - -(43) Project [codegen id : 1] -Output [1]: [ss_net_paid#51] -Input [2]: [ss_quantity#37, ss_net_paid#51] - -(44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#51] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [2]: [sum#71, count#72] -Results [2]: [sum#73, count#74] - -(45) Exchange -Input [2]: [sum#73, count#74] -Arguments: SinglePartition, true, [id=#75] - -(46) HashAggregate [codegen id : 2] -Input [2]: [sum#73, count#74] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#51))#76] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#51))#76 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#77] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] -* HashAggregate (53) -+- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * Filter (49) - +- * ColumnarToRow (48) - +- Scan parquet default.store_sales (47) - - -(47) Scan parquet default.store_sales -Output [1]: [ss_quantity#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(48) ColumnarToRow [codegen id : 1] -Input [1]: [ss_quantity#37] - -(49) Filter [codegen id : 1] -Input [1]: [ss_quantity#37] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) - -(50) Project [codegen id : 1] -Output: [] -Input [1]: [ss_quantity#37] - -(51) HashAggregate [codegen id : 1] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#78] -Results [1]: [count#79] - -(52) Exchange -Input [1]: [count#79] -Arguments: SinglePartition, true, [id=#80] - -(53) HashAggregate [codegen id : 2] -Input [1]: [count#79] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#81] -Results [1]: [count(1)#81 AS count(1)#82] - -Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] -* HashAggregate (60) -+- Exchange (59) - +- * HashAggregate (58) - +- * Project (57) - +- * Filter (56) - +- * ColumnarToRow (55) - +- Scan parquet default.store_sales (54) - - -(54) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_ext_discount_amt#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(55) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] - -(56) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) - -(57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#43] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] - -(58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#43] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#43))] -Aggregate Attributes [2]: [sum#83, count#84] -Results [2]: [sum#85, count#86] - -(59) Exchange -Input [2]: [sum#85, count#86] -Arguments: SinglePartition, true, [id=#87] - -(60) HashAggregate [codegen id : 2] -Input [2]: [sum#85, count#86] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#43))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#43))#88] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#43))#88 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#89] - -Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * Filter (63) - +- * ColumnarToRow (62) - +- Scan parquet default.store_sales (61) - - -(61) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_net_paid#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(62) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_net_paid#51] - -(63) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_net_paid#51] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) - -(64) Project [codegen id : 1] -Output [1]: [ss_net_paid#51] -Input [2]: [ss_quantity#37, ss_net_paid#51] - -(65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#51] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [2]: [sum#90, count#91] -Results [2]: [sum#92, count#93] - -(66) Exchange -Input [2]: [sum#92, count#93] -Arguments: SinglePartition, true, [id=#94] - -(67) HashAggregate [codegen id : 2] -Input [2]: [sum#92, count#93] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#51))#95] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#51))#95 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#96] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] -* HashAggregate (74) -+- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * Filter (70) - +- * ColumnarToRow (69) - +- Scan parquet default.store_sales (68) - - -(68) Scan parquet default.store_sales -Output [1]: [ss_quantity#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(69) ColumnarToRow [codegen id : 1] -Input [1]: [ss_quantity#37] - -(70) Filter [codegen id : 1] -Input [1]: [ss_quantity#37] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) - -(71) Project [codegen id : 1] -Output: [] -Input [1]: [ss_quantity#37] - -(72) HashAggregate [codegen id : 1] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#97] -Results [1]: [count#98] - -(73) Exchange -Input [1]: [count#98] -Arguments: SinglePartition, true, [id=#99] - -(74) HashAggregate [codegen id : 2] -Input [1]: [count#98] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#100] -Results [1]: [count(1)#100 AS count(1)#101] - -Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] -* HashAggregate (81) -+- Exchange (80) - +- * HashAggregate (79) - +- * Project (78) - +- * Filter (77) - +- * ColumnarToRow (76) - +- Scan parquet default.store_sales (75) - - -(75) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_ext_discount_amt#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(76) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] - -(77) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) - -(78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#43] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] - -(79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#43] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#43))] -Aggregate Attributes [2]: [sum#102, count#103] -Results [2]: [sum#104, count#105] - -(80) Exchange -Input [2]: [sum#104, count#105] -Arguments: SinglePartition, true, [id=#106] - -(81) HashAggregate [codegen id : 2] -Input [2]: [sum#104, count#105] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#43))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#43))#107] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#43))#107 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#108] - -Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* HashAggregate (88) -+- Exchange (87) - +- * HashAggregate (86) - +- * Project (85) - +- * Filter (84) - +- * ColumnarToRow (83) - +- Scan parquet default.store_sales (82) - - -(82) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_net_paid#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(83) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_net_paid#51] - -(84) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_net_paid#51] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) - -(85) Project [codegen id : 1] -Output [1]: [ss_net_paid#51] -Input [2]: [ss_quantity#37, ss_net_paid#51] - -(86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#51] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [2]: [sum#109, count#110] -Results [2]: [sum#111, count#112] - -(87) Exchange -Input [2]: [sum#111, count#112] -Arguments: SinglePartition, true, [id=#113] - -(88) HashAggregate [codegen id : 2] -Input [2]: [sum#111, count#112] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#51))#114] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#51))#114 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#115] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* HashAggregate (95) -+- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * Filter (91) - +- * ColumnarToRow (90) - +- Scan parquet default.store_sales (89) - - -(89) Scan parquet default.store_sales -Output [1]: [ss_quantity#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(90) ColumnarToRow [codegen id : 1] -Input [1]: [ss_quantity#37] - -(91) Filter [codegen id : 1] -Input [1]: [ss_quantity#37] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) - -(92) Project [codegen id : 1] -Output: [] -Input [1]: [ss_quantity#37] - -(93) HashAggregate [codegen id : 1] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#116] -Results [1]: [count#117] - -(94) Exchange -Input [1]: [count#117] -Arguments: SinglePartition, true, [id=#118] - -(95) HashAggregate [codegen id : 2] -Input [1]: [count#117] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#119] -Results [1]: [count(1)#119 AS count(1)#120] - -Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] -* HashAggregate (102) -+- Exchange (101) - +- * HashAggregate (100) - +- * Project (99) - +- * Filter (98) - +- * ColumnarToRow (97) - +- Scan parquet default.store_sales (96) - - -(96) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_ext_discount_amt#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(97) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] - -(98) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) - -(99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#43] -Input [2]: [ss_quantity#37, ss_ext_discount_amt#43] - -(100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#43] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#43))] -Aggregate Attributes [2]: [sum#121, count#122] -Results [2]: [sum#123, count#124] - -(101) Exchange -Input [2]: [sum#123, count#124] -Arguments: SinglePartition, true, [id=#125] - -(102) HashAggregate [codegen id : 2] -Input [2]: [sum#123, count#124] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#43))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#43))#126] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#43))#126 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#127] - -Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] -* HashAggregate (109) -+- Exchange (108) - +- * HashAggregate (107) - +- * Project (106) - +- * Filter (105) - +- * ColumnarToRow (104) - +- Scan parquet default.store_sales (103) - - -(103) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_net_paid#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(104) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_net_paid#51] - -(105) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_net_paid#51] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) - -(106) Project [codegen id : 1] -Output [1]: [ss_net_paid#51] -Input [2]: [ss_quantity#37, ss_net_paid#51] - -(107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#51] -Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [2]: [sum#128, count#129] -Results [2]: [sum#130, count#131] - -(108) Exchange -Input [2]: [sum#130, count#131] -Arguments: SinglePartition, true, [id=#132] - -(109) HashAggregate [codegen id : 2] -Input [2]: [sum#130, count#131] -Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#51))#133] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#51))#133 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#134] - - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q9/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q9/simplified.txt deleted file mode 100644 index 66502080d..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q9/simplified.txt +++ /dev/null @@ -1,186 +0,0 @@ -WholeStageCodegen (1) - Project - Subquery #1 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity] - Subquery #2 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #2 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #3 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid] - Subquery #4 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #4 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity] - Subquery #5 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #5 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] - Subquery #6 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #6 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid] - Subquery #7 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #7 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity] - Subquery #8 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #8 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] - Subquery #9 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #9 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid] - Subquery #10 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #10 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity] - Subquery #11 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #11 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] - Subquery #12 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #12 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid] - Subquery #13 - WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #13 - WholeStageCodegen (1) - HashAggregate [count,count] - Project - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity] - Subquery #14 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] - InputAdapter - Exchange #14 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] - Project [ss_ext_discount_amt] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] - Subquery #15 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] - InputAdapter - Exchange #15 - WholeStageCodegen (1) - HashAggregate [ss_net_paid] [sum,count,sum,count] - Project [ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_net_paid] - Filter [r_reason_sk] - ColumnarToRow - InputAdapter - Scan parquet default.reason [r_reason_sk] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q90/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q90/explain.txt deleted file mode 100644 index 550bf89ce..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q90/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- BroadcastNestedLoopJoin Inner BuildRight (48) - :- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.web_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.household_demographics (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.time_dim (11) - : +- BroadcastExchange (22) - : +- * Project (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.web_page (18) - +- BroadcastExchange (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet default.web_sales (28) - : : +- ReusedExchange (31) - : +- BroadcastExchange (38) - : +- * Project (37) - : +- * Filter (36) - : +- * ColumnarToRow (35) - : +- Scan parquet default.time_dim (34) - +- ReusedExchange (41) - - -(1) Scan parquet default.web_sales -Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(4) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#4, hd_dep_count#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [hd_demo_sk#4, hd_dep_count#5] - -(6) Filter [codegen id : 1] -Input [2]: [hd_demo_sk#4, hd_dep_count#5] -Condition : ((isnotnull(hd_dep_count#5) AND (hd_dep_count#5 = 6)) AND isnotnull(hd_demo_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [hd_demo_sk#4] -Input [2]: [hd_demo_sk#4, hd_dep_count#5] - -(8) BroadcastExchange -Input [1]: [hd_demo_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_ship_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(10) Project [codegen id : 4] -Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#4] - -(11) Scan parquet default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [t_time_sk#7, t_hour#8] - -(13) Filter [codegen id : 2] -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) - -(14) Project [codegen id : 2] -Output [1]: [t_time_sk#7] -Input [2]: [t_time_sk#7, t_hour#8] - -(15) BroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#7] -Join condition: None - -(17) Project [codegen id : 4] -Output [1]: [ws_web_page_sk#3] -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] - -(18) Scan parquet default.web_page -Output [2]: [wp_web_page_sk#10, wp_char_count#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [2]: [wp_web_page_sk#10, wp_char_count#11] - -(20) Filter [codegen id : 3] -Input [2]: [wp_web_page_sk#10, wp_char_count#11] -Condition : (((isnotnull(wp_char_count#11) AND (wp_char_count#11 >= 5000)) AND (wp_char_count#11 <= 5200)) AND isnotnull(wp_web_page_sk#10)) - -(21) Project [codegen id : 3] -Output [1]: [wp_web_page_sk#10] -Input [2]: [wp_web_page_sk#10, wp_char_count#11] - -(22) BroadcastExchange -Input [1]: [wp_web_page_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#10] -Join condition: None - -(24) Project [codegen id : 4] -Output: [] -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#10] - -(25) HashAggregate [codegen id : 4] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#13] -Results [1]: [count#14] - -(26) Exchange -Input [1]: [count#14] -Arguments: SinglePartition, true, [id=#15] - -(27) HashAggregate [codegen id : 5] -Input [1]: [count#14] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [1]: [count(1)#16 AS amc#17] - -(28) Scan parquet default.web_sales -Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 9] -Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(30) Filter [codegen id : 9] -Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(31) ReusedExchange [Reuses operator id: 8] -Output [1]: [hd_demo_sk#4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_ship_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(33) Project [codegen id : 9] -Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#4] - -(34) Scan parquet default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 7] -Input [2]: [t_time_sk#7, t_hour#8] - -(36) Filter [codegen id : 7] -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 19)) AND (t_hour#8 <= 20)) AND isnotnull(t_time_sk#7)) - -(37) Project [codegen id : 7] -Output [1]: [t_time_sk#7] -Input [2]: [t_time_sk#7, t_hour#8] - -(38) BroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#7] -Join condition: None - -(40) Project [codegen id : 9] -Output [1]: [ws_web_page_sk#3] -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] - -(41) ReusedExchange [Reuses operator id: 22] -Output [1]: [wp_web_page_sk#10] - -(42) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#10] -Join condition: None - -(43) Project [codegen id : 9] -Output: [] -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#10] - -(44) HashAggregate [codegen id : 9] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#19] -Results [1]: [count#20] - -(45) Exchange -Input [1]: [count#20] -Arguments: SinglePartition, true, [id=#21] - -(46) HashAggregate [codegen id : 10] -Input [1]: [count#20] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [1]: [count(1)#22 AS pmc#23] - -(47) BroadcastExchange -Input [1]: [pmc#23] -Arguments: IdentityBroadcastMode, [id=#24] - -(48) BroadcastNestedLoopJoin -Join condition: None - -(49) Project [codegen id : 11] -Output [1]: [CheckOverflow((promote_precision(cast(amc#17 as decimal(15,4))) / promote_precision(cast(pmc#23 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#25] -Input [2]: [amc#17, pmc#23] - -(50) TakeOrderedAndProject -Input [1]: [am_pm_ratio#25] -Arguments: 100, [am_pm_ratio#25 ASC NULLS FIRST], [am_pm_ratio#25] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q90/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q90/simplified.txt deleted file mode 100644 index 121d84d9d..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q90/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [am_pm_ratio] - WholeStageCodegen (11) - Project [amc,pmc] - InputAdapter - BroadcastNestedLoopJoin - WholeStageCodegen (5) - HashAggregate [count] [count(1),amc,count] - InputAdapter - Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] - Filter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_dep_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [t_time_sk] - Filter [t_hour,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [wp_web_page_sk] - Filter [wp_char_count,wp_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_page [wp_web_page_sk,wp_char_count] - BroadcastExchange #5 - WholeStageCodegen (10) - HashAggregate [count] [count(1),pmc,count] - InputAdapter - Exchange #6 - WholeStageCodegen (9) - HashAggregate [count,count] - Project - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] - Filter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Project [t_time_sk] - Filter [t_hour,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour] - InputAdapter - ReusedExchange [wp_web_page_sk] #4 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q91/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q91/explain.txt deleted file mode 100644 index 1956baf78..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q91/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -* Sort (47) -+- Exchange (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * Project (16) - : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet default.call_center (1) - : : : : : +- BroadcastExchange (7) - : : : : : +- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.catalog_returns (4) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * Filter (12) - : : : : +- * ColumnarToRow (11) - : : : : +- Scan parquet default.date_dim (10) - : : : +- BroadcastExchange (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.customer (17) - : : +- BroadcastExchange (27) - : : +- * Project (26) - : : +- * Filter (25) - : : +- * ColumnarToRow (24) - : : +- Scan parquet default.customer_address (23) - : +- BroadcastExchange (33) - : +- * Filter (32) - : +- * ColumnarToRow (31) - : +- Scan parquet default.customer_demographics (30) - +- BroadcastExchange (40) - +- * Project (39) - +- * Filter (38) - +- * ColumnarToRow (37) - +- Scan parquet default.household_demographics (36) - - -(1) Scan parquet default.call_center -Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] - -(3) Filter [codegen id : 7] -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Condition : isnotnull(cc_call_center_sk#1) - -(4) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#5, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [cr_returned_date_sk#5, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8] - -(6) Filter [codegen id : 1] -Input [4]: [cr_returned_date_sk#5, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8] -Condition : ((isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returned_date_sk#5)) AND isnotnull(cr_returning_customer_sk#6)) - -(7) BroadcastExchange -Input [4]: [cr_returned_date_sk#5, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, false] as bigint)),false), [id=#9] - -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cc_call_center_sk#1] -Right keys [1]: [cr_call_center_sk#7] -Join condition: None - -(9) Project [codegen id : 7] -Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returned_date_sk#5, cr_returning_customer_sk#6, cr_net_loss#8] -Input [8]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returned_date_sk#5, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8] - -(10) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_moy#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] - -(12) Filter [codegen id : 2] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 1998)) AND (d_moy#12 = 11)) AND isnotnull(d_date_sk#10)) - -(13) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#11, d_moy#12] - -(14) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(15) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returned_date_sk#5] -Right keys [1]: [d_date_sk#10] -Join condition: None - -(16) Project [codegen id : 7] -Output [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] -Input [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returned_date_sk#5, cr_returning_customer_sk#6, cr_net_loss#8, d_date_sk#10] - -(17) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(19) Filter [codegen id : 3] -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Condition : (((isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#17)) AND isnotnull(c_current_cdemo_sk#15)) AND isnotnull(c_current_hdemo_sk#16)) - -(20) BroadcastExchange -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] - -(21) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returning_customer_sk#6] -Right keys [1]: [c_customer_sk#14] -Join condition: None - -(22) Project [codegen id : 7] -Output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(23) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_gmt_offset#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] - -(25) Filter [codegen id : 4] -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Condition : ((isnotnull(ca_gmt_offset#20) AND (ca_gmt_offset#20 = -7.00)) AND isnotnull(ca_address_sk#19)) - -(26) Project [codegen id : 4] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] - -(27) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] - -(28) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#19] -Join condition: None - -(29) Project [codegen id : 7] -Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] -Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17, ca_address_sk#19] - -(30) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -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)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 5] -Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] - -(32) Filter [codegen id : 5] -Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -Condition : ((((cd_marital_status#23 = M) AND (cd_education_status#24 = Unknown)) OR ((cd_marital_status#23 = W) AND (cd_education_status#24 = Advanced Degree))) AND isnotnull(cd_demo_sk#22)) - -(33) BroadcastExchange -Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] - -(34) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#15] -Right keys [1]: [cd_demo_sk#22] -Join condition: None - -(35) Project [codegen id : 7] -Output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] -Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] - -(36) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#26, hd_buy_potential#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(37) ColumnarToRow [codegen id : 6] -Input [2]: [hd_demo_sk#26, hd_buy_potential#27] - -(38) Filter [codegen id : 6] -Input [2]: [hd_demo_sk#26, hd_buy_potential#27] -Condition : ((isnotnull(hd_buy_potential#27) AND StartsWith(hd_buy_potential#27, Unknown)) AND isnotnull(hd_demo_sk#26)) - -(39) Project [codegen id : 6] -Output [1]: [hd_demo_sk#26] -Input [2]: [hd_demo_sk#26, hd_buy_potential#27] - -(40) BroadcastExchange -Input [1]: [hd_demo_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] - -(41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#16] -Right keys [1]: [hd_demo_sk#26] -Join condition: None - -(42) Project [codegen id : 7] -Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] -Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24, hd_demo_sk#26] - -(43) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] -Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] -Aggregate Attributes [1]: [sum#29] -Results [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#30] - -(44) Exchange -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#30] -Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, 5), true, [id=#31] - -(45) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#30] -Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#8))#32] -Results [4]: [cc_call_center_id#2 AS Call_Center#33, cc_name#3 AS Call_Center_Name#34, cc_manager#4 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#8))#32,17,2) AS Returns_Loss#36] - -(46) Exchange -Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] -Arguments: rangepartitioning(Returns_Loss#36 DESC NULLS LAST, 5), true, [id=#37] - -(47) Sort [codegen id : 9] -Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] -Arguments: [Returns_Loss#36 DESC NULLS LAST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q91/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q91/simplified.txt deleted file mode 100644 index 58ebe15d1..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q91/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (9) - Sort [Returns_Loss] - InputAdapter - Exchange [Returns_Loss] #1 - WholeStageCodegen (8) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] - InputAdapter - Exchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - WholeStageCodegen (7) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returned_date_sk,cr_returning_customer_sk,cr_net_loss] - BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - Filter [cc_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Filter [cd_marital_status,cd_education_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q92/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q92/explain.txt deleted file mode 100644 index 8a441392f..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q92/explain.txt +++ /dev/null @@ -1,196 +0,0 @@ -== Physical Plan == -* Sort (34) -+- * HashAggregate (33) - +- Exchange (32) - +- * HashAggregate (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.web_sales (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.item (4) - : +- BroadcastExchange (25) - : +- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.web_sales (11) - : +- BroadcastExchange (18) - : +- * Project (17) - : +- * Filter (16) - : +- * ColumnarToRow (15) - : +- Scan parquet default.date_dim (14) - +- ReusedExchange (28) - - -(1) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_discount_amt#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt), IsNotNull(ws_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_discount_amt#3] - -(3) Filter [codegen id : 6] -Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_discount_amt#3] -Condition : ((isnotnull(ws_item_sk#2) AND isnotnull(ws_ext_discount_amt#3)) AND isnotnull(ws_sold_date_sk#1)) - -(4) Scan parquet default.item -Output [2]: [i_item_sk#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#4, i_manufact_id#5] - -(6) Filter [codegen id : 1] -Input [2]: [i_item_sk#4, i_manufact_id#5] -Condition : ((isnotnull(i_manufact_id#5) AND (i_manufact_id#5 = 350)) AND isnotnull(i_item_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [i_item_sk#4] -Input [2]: [i_item_sk#4, i_manufact_id#5] - -(8) BroadcastExchange -Input [1]: [i_item_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] - -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#4] -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [ws_sold_date_sk#1, ws_ext_discount_amt#3, i_item_sk#4] -Input [4]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_discount_amt#3, i_item_sk#4] - -(11) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_discount_amt#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_discount_amt#3] - -(13) Filter [codegen id : 3] -Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_discount_amt#3] -Condition : (isnotnull(ws_sold_date_sk#1) AND isnotnull(ws_item_sk#2)) - -(14) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#7, d_date#8] - -(16) Filter [codegen id : 2] -Input [2]: [d_date_sk#7, d_date#8] -Condition : (((isnotnull(d_date#8) AND (d_date#8 >= 10983)) AND (d_date#8 <= 11073)) AND isnotnull(d_date_sk#7)) - -(17) Project [codegen id : 2] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] - -(18) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] - -(19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#7] -Join condition: None - -(20) Project [codegen id : 3] -Output [2]: [ws_item_sk#2, ws_ext_discount_amt#3] -Input [4]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_discount_amt#3, d_date_sk#7] - -(21) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#2, ws_ext_discount_amt#3] -Keys [1]: [ws_item_sk#2] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#3))] -Aggregate Attributes [2]: [sum#10, count#11] -Results [3]: [ws_item_sk#2, sum#12, count#13] - -(22) Exchange -Input [3]: [ws_item_sk#2, sum#12, count#13] -Arguments: hashpartitioning(ws_item_sk#2, 5), true, [id=#14] - -(23) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#2, sum#12, count#13] -Keys [1]: [ws_item_sk#2] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#3))#15] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#3))#15 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))#16, ws_item_sk#2 AS ws_item_sk#2#17] - -(24) Filter [codegen id : 4] -Input [2]: [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))#16, ws_item_sk#2#17] -Condition : isnotnull((CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))#16) - -(25) BroadcastExchange -Input [2]: [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))#16, ws_item_sk#2#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#18] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_sk#4] -Right keys [1]: [ws_item_sk#2#17] -Join condition: (cast(ws_ext_discount_amt#3 as decimal(14,7)) > (CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))#16) - -(27) Project [codegen id : 6] -Output [2]: [ws_sold_date_sk#1, ws_ext_discount_amt#3] -Input [5]: [ws_sold_date_sk#1, ws_ext_discount_amt#3, i_item_sk#4, (CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))#16, ws_item_sk#2#17] - -(28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#7] - -(29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#7] -Join condition: None - -(30) Project [codegen id : 6] -Output [1]: [ws_ext_discount_amt#3] -Input [3]: [ws_sold_date_sk#1, ws_ext_discount_amt#3, d_date_sk#7] - -(31) HashAggregate [codegen id : 6] -Input [1]: [ws_ext_discount_amt#3] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#3))] -Aggregate Attributes [1]: [sum#19] -Results [1]: [sum#20] - -(32) Exchange -Input [1]: [sum#20] -Arguments: SinglePartition, true, [id=#21] - -(33) HashAggregate [codegen id : 7] -Input [1]: [sum#20] -Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#3))#22] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#3))#22,17,2) AS Excess Discount Amount #23] - -(34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #23] -Arguments: [Excess Discount Amount #23 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q92/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q92/simplified.txt deleted file mode 100644 index 1f24a7c96..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q92/simplified.txt +++ /dev/null @@ -1,50 +0,0 @@ -WholeStageCodegen (7) - Sort [Excess Discount Amount ] - HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [ws_ext_discount_amt] [sum,sum] - Project [ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,ws_ext_discount_amt] - BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))] - Project [ws_sold_date_sk,ws_ext_discount_amt,i_item_sk] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_discount_amt] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [i_item_sk] - Filter [i_manufact_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (4) - Filter [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6))),ws_item_sk,sum,count] - InputAdapter - Exchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] - Project [ws_item_sk,ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_discount_amt] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #5 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q93/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q93/explain.txt deleted file mode 100644 index 620aa6727..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q93/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * HashAggregate (16) - +- * Project (15) - +- * BroadcastHashJoin Inner BuildRight (14) - :- * Project (8) - : +- * BroadcastHashJoin Inner BuildRight (7) - : :- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (6) - : +- * Filter (5) - : +- * ColumnarToRow (4) - : +- Scan parquet default.store_returns (3) - +- BroadcastExchange (13) - +- * Project (12) - +- * Filter (11) - +- * ColumnarToRow (10) - +- Scan parquet default.reason (9) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] - -(3) Scan parquet default.store_returns -Output [4]: [sr_item_sk#6, sr_reason_sk#7, sr_ticket_number#8, sr_return_quantity#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] -ReadSchema: struct - -(4) ColumnarToRow [codegen id : 1] -Input [4]: [sr_item_sk#6, sr_reason_sk#7, sr_ticket_number#8, sr_return_quantity#9] - -(5) Filter [codegen id : 1] -Input [4]: [sr_item_sk#6, sr_reason_sk#7, sr_ticket_number#8, sr_return_quantity#9] -Condition : ((isnotnull(sr_item_sk#6) AND isnotnull(sr_ticket_number#8)) AND isnotnull(sr_reason_sk#7)) - -(6) BroadcastExchange -Input [4]: [sr_item_sk#6, sr_reason_sk#7, sr_ticket_number#8, sr_return_quantity#9] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[2, bigint, false]),false), [id=#10] - -(7) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#3 as bigint)] -Right keys [2]: [sr_item_sk#6, sr_ticket_number#8] -Join condition: None - -(8) Project [codegen id : 3] -Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#7, sr_return_quantity#9] -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#6, sr_reason_sk#7, sr_ticket_number#8, sr_return_quantity#9] - -(9) Scan parquet default.reason -Output [2]: [r_reason_sk#11, r_reason_desc#12] -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)] -ReadSchema: struct - -(10) ColumnarToRow [codegen id : 2] -Input [2]: [r_reason_sk#11, r_reason_desc#12] - -(11) Filter [codegen id : 2] -Input [2]: [r_reason_sk#11, r_reason_desc#12] -Condition : ((isnotnull(r_reason_desc#12) AND (r_reason_desc#12 = reason 28)) AND isnotnull(r_reason_sk#11)) - -(12) Project [codegen id : 2] -Output [1]: [r_reason_sk#11] -Input [2]: [r_reason_sk#11, r_reason_desc#12] - -(13) BroadcastExchange -Input [1]: [r_reason_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [sr_reason_sk#7] -Right keys [1]: [cast(r_reason_sk#11 as bigint)] -Join condition: None - -(15) Project [codegen id : 3] -Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#9) THEN CheckOverflow((promote_precision(cast(cast((ss_quantity#4 - sr_return_quantity#9) as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2), true) ELSE CheckOverflow((promote_precision(cast(cast(ss_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2), true) END AS act_sales#14] -Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#7, sr_return_quantity#9, r_reason_sk#11] - -(16) HashAggregate [codegen id : 3] -Input [2]: [ss_customer_sk#2, act_sales#14] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [partial_sum(act_sales#14)] -Aggregate Attributes [2]: [sum#15, isEmpty#16] -Results [3]: [ss_customer_sk#2, sum#17, isEmpty#18] - -(17) Exchange -Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#19] - -(18) HashAggregate [codegen id : 4] -Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [sum(act_sales#14)] -Aggregate Attributes [1]: [sum(act_sales#14)#20] -Results [2]: [ss_customer_sk#2, sum(act_sales#14)#20 AS sumsales#21] - -(19) TakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#21] -Arguments: 100, [sumsales#21 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#21] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q93/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q93/simplified.txt deleted file mode 100644 index 81de31a44..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q93/simplified.txt +++ /dev/null @@ -1,29 +0,0 @@ -TakeOrderedAndProject [sumsales,ss_customer_sk] - WholeStageCodegen (4) - HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] - InputAdapter - Exchange [ss_customer_sk] #1 - WholeStageCodegen (3) - HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] - Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] - BroadcastHashJoin [sr_reason_sk,r_reason_sk] - Project [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number,sr_reason_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [r_reason_sk] - Filter [r_reason_desc,r_reason_sk] - ColumnarToRow - InputAdapter - Scan parquet default.reason [r_reason_sk,r_reason_desc] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q94/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q94/explain.txt deleted file mode 100644 index 2abbe4f9b..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q94/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* Sort (41) -+- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * BroadcastHashJoin LeftAnti BuildRight (13) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Project (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.web_sales (4) - : : : +- BroadcastExchange (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.web_returns (10) - : : +- BroadcastExchange (18) - : : +- * Project (17) - : : +- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet default.date_dim (14) - : +- BroadcastExchange (25) - : +- * Project (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet default.customer_address (21) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- * ColumnarToRow (29) - +- Scan parquet default.web_site (28) - - -(1) Scan parquet default.web_sales -Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(3) Filter [codegen id : 6] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(4) Scan parquet default.web_sales -Output [2]: [ws_warehouse_sk#4, ws_order_number#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [ws_warehouse_sk#4, ws_order_number#5] - -(6) Project [codegen id : 1] -Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#8, ws_order_number#5 AS ws_order_number#5#9] -Input [2]: [ws_warehouse_sk#4, ws_order_number#5] - -(7) BroadcastExchange -Input [2]: [ws_warehouse_sk#4#8, ws_order_number#5#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#10] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_order_number#5] -Right keys [1]: [ws_order_number#5#9] -Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#8) - -(9) Project [codegen id : 6] -Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(10) Scan parquet default.web_returns -Output [1]: [wr_order_number#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [1]: [wr_order_number#11] - -(12) BroadcastExchange -Input [1]: [wr_order_number#11] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [id=#12] - -(13) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cast(ws_order_number#5 as bigint)] -Right keys [1]: [wr_order_number#11] -Join condition: None - -(14) Scan parquet default.date_dim -Output [2]: [d_date_sk#13, d_date#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#13, d_date#14] - -(16) Filter [codegen id : 3] -Input [2]: [d_date_sk#13, d_date#14] -Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 10623)) AND (d_date#14 <= 10683)) AND isnotnull(d_date_sk#13)) - -(17) Project [codegen id : 3] -Output [1]: [d_date_sk#13] -Input [2]: [d_date_sk#13, d_date#14] - -(18) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] - -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#13] -Join condition: None - -(20) Project [codegen id : 6] -Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#13] - -(21) Scan parquet default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(22) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#16, ca_state#17] - -(23) Filter [codegen id : 4] -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = IL)) AND isnotnull(ca_address_sk#16)) - -(24) Project [codegen id : 4] -Output [1]: [ca_address_sk#16] -Input [2]: [ca_address_sk#16, ca_state#17] - -(25) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#16] -Join condition: None - -(27) Project [codegen id : 6] -Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] - -(28) Scan parquet default.web_site -Output [2]: [web_site_sk#19, web_company_name#20] -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)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 5] -Input [2]: [web_site_sk#19, web_company_name#20] - -(30) Filter [codegen id : 5] -Input [2]: [web_site_sk#19, web_company_name#20] -Condition : ((isnotnull(web_company_name#20) AND (web_company_name#20 = pri)) AND isnotnull(web_site_sk#19)) - -(31) Project [codegen id : 5] -Output [1]: [web_site_sk#19] -Input [2]: [web_site_sk#19, web_company_name#20] - -(32) BroadcastExchange -Input [1]: [web_site_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] - -(33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#19] -Join condition: None - -(34) Project [codegen id : 6] -Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#19] - -(35) HashAggregate [codegen id : 6] -Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Keys [1]: [ws_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#24, sum#25] - -(36) Exchange -Input [3]: [ws_order_number#5, sum#24, sum#25] -Arguments: hashpartitioning(ws_order_number#5, 5), true, [id=#26] - -(37) HashAggregate [codegen id : 7] -Input [3]: [ws_order_number#5, sum#24, sum#25] -Keys [1]: [ws_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#24, sum#25] - -(38) HashAggregate [codegen id : 7] -Input [3]: [ws_order_number#5, sum#24, sum#25] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#27] -Results [3]: [sum#24, sum#25, count#28] - -(39) Exchange -Input [3]: [sum#24, sum#25, count#28] -Arguments: SinglePartition, true, [id=#29] - -(40) HashAggregate [codegen id : 8] -Input [3]: [sum#24, sum#25, count#28] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#27] -Results [3]: [count(ws_order_number#5)#27 AS order count #30, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#22,17,2) AS total shipping cost #31, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#23,17,2) AS total net profit #32] - -(41) Sort [codegen id : 8] -Input [3]: [order count #30, total shipping cost #31, total net profit #32] -Arguments: [order count #30 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q94/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q94/simplified.txt deleted file mode 100644 index 5e7d7db5c..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q94/simplified.txt +++ /dev/null @@ -1,62 +0,0 @@ -WholeStageCodegen (8) - Sort [order count ] - HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (7) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (6) - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - BroadcastHashJoin [ws_order_number,wr_order_number] - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [ws_warehouse_sk,ws_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Project [web_site_sk] - Filter [web_company_name,web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_site [web_site_sk,web_company_name] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q95/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q95/explain.txt deleted file mode 100644 index 1cc99e296..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q95/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* Sort (56) -+- * HashAggregate (55) - +- Exchange (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- Exchange (51) - +- * HashAggregate (50) - +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * BroadcastHashJoin LeftSemi BuildRight (28) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.web_sales (1) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.web_sales (4) - : : : : +- BroadcastExchange (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.web_sales (7) - : : : +- BroadcastExchange (27) - : : : +- * Project (26) - : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : :- * Filter (17) - : : : : +- * ColumnarToRow (16) - : : : : +- Scan parquet default.web_returns (15) - : : : +- BroadcastExchange (24) - : : : +- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Filter (20) - : : : : +- * ColumnarToRow (19) - : : : : +- Scan parquet default.web_sales (18) - : : : +- ReusedExchange (21) - : : +- BroadcastExchange (33) - : : +- * Project (32) - : : +- * Filter (31) - : : +- * ColumnarToRow (30) - : : +- Scan parquet default.date_dim (29) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.customer_address (36) - +- BroadcastExchange (47) - +- * Project (46) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet default.web_site (43) - - -(1) Scan parquet default.web_sales -Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 9] -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(3) Filter [codegen id : 9] -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(4) Scan parquet default.web_sales -Output [2]: [ws_warehouse_sk#7, ws_order_number#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ws_warehouse_sk#7, ws_order_number#4] - -(6) Filter [codegen id : 2] -Input [2]: [ws_warehouse_sk#7, ws_order_number#4] -Condition : (isnotnull(ws_order_number#4) AND isnotnull(ws_warehouse_sk#7)) - -(7) Scan parquet default.web_sales -Output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 1] -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] - -(9) Filter [codegen id : 1] -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) - -(10) BroadcastExchange -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#10] - -(11) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#9] -Join condition: NOT (ws_warehouse_sk#7 = ws_warehouse_sk#8) - -(12) Project [codegen id : 2] -Output [1]: [ws_order_number#4 AS ws_order_number#4#11] -Input [4]: [ws_warehouse_sk#7, ws_order_number#4, ws_warehouse_sk#8, ws_order_number#9] - -(13) BroadcastExchange -Input [1]: [ws_order_number#4#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(14) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#4#11] -Join condition: None - -(15) Scan parquet default.web_returns -Output [1]: [wr_order_number#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] -ReadSchema: struct - -(16) ColumnarToRow [codegen id : 5] -Input [1]: [wr_order_number#13] - -(17) Filter [codegen id : 5] -Input [1]: [wr_order_number#13] -Condition : isnotnull(wr_order_number#13) - -(18) Scan parquet default.web_sales -Output [2]: [ws_warehouse_sk#7, ws_order_number#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 4] -Input [2]: [ws_warehouse_sk#7, ws_order_number#4] - -(20) Filter [codegen id : 4] -Input [2]: [ws_warehouse_sk#7, ws_order_number#4] -Condition : (isnotnull(ws_order_number#4) AND isnotnull(ws_warehouse_sk#7)) - -(21) ReusedExchange [Reuses operator id: 10] -Output [2]: [ws_warehouse_sk#14, ws_order_number#15] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#15] -Join condition: NOT (ws_warehouse_sk#7 = ws_warehouse_sk#14) - -(23) Project [codegen id : 4] -Output [1]: [ws_order_number#4] -Input [4]: [ws_warehouse_sk#7, ws_order_number#4, ws_warehouse_sk#14, ws_order_number#15] - -(24) BroadcastExchange -Input [1]: [ws_order_number#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [wr_order_number#13] -Right keys [1]: [cast(ws_order_number#4 as bigint)] -Join condition: None - -(26) Project [codegen id : 5] -Output [1]: [wr_order_number#13] -Input [2]: [wr_order_number#13, ws_order_number#4] - -(27) BroadcastExchange -Input [1]: [wr_order_number#13] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [id=#17] - -(28) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#13] -Join condition: None - -(29) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#18, d_date#19] - -(31) Filter [codegen id : 6] -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 10623)) AND (d_date#19 <= 10683)) AND isnotnull(d_date_sk#18)) - -(32) Project [codegen id : 6] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_date#19] - -(33) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] - -(34) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#18] -Join condition: None - -(35) Project [codegen id : 9] -Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#18] - -(36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(37) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_state#22] - -(38) Filter [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : ((isnotnull(ca_state#22) AND (ca_state#22 = IL)) AND isnotnull(ca_address_sk#21)) - -(39) Project [codegen id : 7] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_state#22] - -(40) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#21] -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] - -(43) Scan parquet default.web_site -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)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 8] -Input [2]: [web_site_sk#24, web_company_name#25] - -(45) Filter [codegen id : 8] -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)) - -(46) Project [codegen id : 8] -Output [1]: [web_site_sk#24] -Input [2]: [web_site_sk#24, web_company_name#25] - -(47) BroadcastExchange -Input [1]: [web_site_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] - -(48) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#24] -Join condition: None - -(49) Project [codegen id : 9] -Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#24] - -(50) HashAggregate [codegen id : 9] -Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Keys [1]: [ws_order_number#4] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] -Results [3]: [ws_order_number#4, sum#29, sum#30] - -(51) Exchange -Input [3]: [ws_order_number#4, sum#29, sum#30] -Arguments: hashpartitioning(ws_order_number#4, 5), true, [id=#31] - -(52) HashAggregate [codegen id : 10] -Input [3]: [ws_order_number#4, sum#29, sum#30] -Keys [1]: [ws_order_number#4] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] -Results [3]: [ws_order_number#4, sum#29, sum#30] - -(53) HashAggregate [codegen id : 10] -Input [3]: [ws_order_number#4, sum#29, sum#30] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#32] -Results [3]: [sum#29, sum#30, count#33] - -(54) Exchange -Input [3]: [sum#29, sum#30, count#33] -Arguments: SinglePartition, true, [id=#34] - -(55) HashAggregate [codegen id : 11] -Input [3]: [sum#29, sum#30, count#33] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#32] -Results [3]: [count(ws_order_number#4)#32 AS order count #35, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#27,17,2) AS total shipping cost #36, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#28,17,2) AS total net profit #37] - -(56) Sort [codegen id : 11] -Input [3]: [order count #35, total shipping cost #36, total net profit #37] -Arguments: [order count #35 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q95/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q95/simplified.txt deleted file mode 100644 index 191ff22c1..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q95/simplified.txt +++ /dev/null @@ -1,84 +0,0 @@ -WholeStageCodegen (11) - Sort [order count ] - HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (10) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (9) - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - BroadcastHashJoin [ws_order_number,wr_order_number] - BroadcastHashJoin [ws_order_number,ws_order_number] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [ws_order_number] - BroadcastHashJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - Project [wr_order_number] - BroadcastHashJoin [wr_order_number,ws_order_number] - Filter [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Project [ws_order_number] - BroadcastHashJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Project [web_site_sk] - Filter [web_company_name,web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_site [web_site_sk,web_company_name] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q96/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q96/explain.txt deleted file mode 100644 index 6729910d9..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q96/explain.txt +++ /dev/null @@ -1,160 +0,0 @@ -== Physical Plan == -* Sort (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * BroadcastHashJoin Inner BuildRight (23) - :- * Project (17) - : +- * BroadcastHashJoin Inner BuildRight (16) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.household_demographics (4) - : +- BroadcastExchange (15) - : +- * Project (14) - : +- * Filter (13) - : +- * ColumnarToRow (12) - : +- Scan parquet default.time_dim (11) - +- BroadcastExchange (22) - +- * Project (21) - +- * Filter (20) - +- * ColumnarToRow (19) - +- Scan parquet default.store (18) - - -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#4, hd_dep_count#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [hd_demo_sk#4, hd_dep_count#5] - -(6) Filter [codegen id : 1] -Input [2]: [hd_demo_sk#4, hd_dep_count#5] -Condition : ((isnotnull(hd_dep_count#5) AND (hd_dep_count#5 = 7)) AND isnotnull(hd_demo_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [hd_demo_sk#4] -Input [2]: [hd_demo_sk#4, hd_dep_count#5] - -(8) BroadcastExchange -Input [1]: [hd_demo_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#4] -Join condition: None - -(10) Project [codegen id : 4] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#4] - -(11) Scan parquet default.time_dim -Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] - -(13) Filter [codegen id : 2] -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) - -(14) Project [codegen id : 2] -Output [1]: [t_time_sk#7] -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] - -(15) BroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#7] -Join condition: None - -(17) Project [codegen id : 4] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] - -(18) Scan parquet default.store -Output [2]: [s_store_sk#11, s_store_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#11, s_store_name#12] - -(20) Filter [codegen id : 3] -Input [2]: [s_store_sk#11, s_store_name#12] -Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) - -(21) Project [codegen id : 3] -Output [1]: [s_store_sk#11] -Input [2]: [s_store_sk#11, s_store_name#12] - -(22) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#11] -Join condition: None - -(24) Project [codegen id : 4] -Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#11] - -(25) HashAggregate [codegen id : 4] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [1]: [count#15] - -(26) Exchange -Input [1]: [count#15] -Arguments: SinglePartition, true, [id=#16] - -(27) HashAggregate [codegen id : 5] -Input [1]: [count#15] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [1]: [count(1)#17 AS count(1)#18] - -(28) Sort [codegen id : 5] -Input [1]: [count(1)#18] -Arguments: [count(1)#18 ASC NULLS FIRST], true, 0 - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q96/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q96/simplified.txt deleted file mode 100644 index 45400b6c5..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q96/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (5) - Sort [count(1)] - HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_dep_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [t_time_sk] - Filter [t_hour,t_minute,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [s_store_sk] - Filter [s_store_name,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q97/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q97/explain.txt deleted file mode 100644 index e904ad94d..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q97/explain.txt +++ /dev/null @@ -1,174 +0,0 @@ -== Physical Plan == -* HashAggregate (29) -+- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- SortMergeJoin FullOuter (25) - :- * Sort (14) - : +- * HashAggregate (13) - : +- Exchange (12) - : +- * HashAggregate (11) - : +- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (24) - +- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Filter (17) - : +- * ColumnarToRow (16) - : +- Scan parquet default.catalog_sales (15) - +- ReusedExchange (18) - - -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] - -(3) Filter [codegen id : 2] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] -Condition : isnotnull(ss_sold_date_sk#1) - -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_month_seq#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] -Condition : (((isnotnull(d_month_seq#5) AND (d_month_seq#5 >= 1200)) AND (d_month_seq#5 <= 1211)) AND isnotnull(d_date_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(10) Project [codegen id : 2] -Output [2]: [ss_item_sk#2, ss_customer_sk#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, d_date_sk#4] - -(11) HashAggregate [codegen id : 2] -Input [2]: [ss_item_sk#2, ss_customer_sk#3] -Keys [2]: [ss_customer_sk#3, ss_item_sk#2] -Functions: [] -Aggregate Attributes: [] -Results [2]: [ss_customer_sk#3, ss_item_sk#2] - -(12) Exchange -Input [2]: [ss_customer_sk#3, ss_item_sk#2] -Arguments: hashpartitioning(ss_customer_sk#3, ss_item_sk#2, 5), true, [id=#7] - -(13) HashAggregate [codegen id : 3] -Input [2]: [ss_customer_sk#3, ss_item_sk#2] -Keys [2]: [ss_customer_sk#3, ss_item_sk#2] -Functions: [] -Aggregate Attributes: [] -Results [2]: [ss_customer_sk#3 AS customer_sk#8, ss_item_sk#2 AS item_sk#9] - -(14) Sort [codegen id : 3] -Input [2]: [customer_sk#8, item_sk#9] -Arguments: [customer_sk#8 ASC NULLS FIRST, item_sk#9 ASC NULLS FIRST], false, 0 - -(15) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#10, cs_bill_customer_sk#11, cs_item_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] -ReadSchema: struct - -(16) ColumnarToRow [codegen id : 5] -Input [3]: [cs_sold_date_sk#10, cs_bill_customer_sk#11, cs_item_sk#12] - -(17) Filter [codegen id : 5] -Input [3]: [cs_sold_date_sk#10, cs_bill_customer_sk#11, cs_item_sk#12] -Condition : isnotnull(cs_sold_date_sk#10) - -(18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#4] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(20) Project [codegen id : 5] -Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_sold_date_sk#10, cs_bill_customer_sk#11, cs_item_sk#12, d_date_sk#4] - -(21) HashAggregate [codegen id : 5] -Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Functions: [] -Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] - -(22) Exchange -Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), true, [id=#13] - -(23) HashAggregate [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Functions: [] -Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#14, cs_item_sk#12 AS item_sk#15] - -(24) Sort [codegen id : 6] -Input [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST], false, 0 - -(25) SortMergeJoin -Left keys [2]: [customer_sk#8, item_sk#9] -Right keys [2]: [customer_sk#14, item_sk#15] -Join condition: None - -(26) Project [codegen id : 7] -Output [2]: [customer_sk#8, customer_sk#14] -Input [4]: [customer_sk#8, item_sk#9, customer_sk#14, item_sk#15] - -(27) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#8, customer_sk#14] -Keys: [] -Functions [3]: [partial_sum(cast(CASE WHEN (isnotnull(customer_sk#8) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (isnull(customer_sk#8) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (isnotnull(customer_sk#8) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [3]: [sum#16, sum#17, sum#18] -Results [3]: [sum#19, sum#20, sum#21] - -(28) Exchange -Input [3]: [sum#19, sum#20, sum#21] -Arguments: SinglePartition, true, [id=#22] - -(29) HashAggregate [codegen id : 8] -Input [3]: [sum#19, sum#20, sum#21] -Keys: [] -Functions [3]: [sum(cast(CASE WHEN (isnotnull(customer_sk#8) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (isnull(customer_sk#8) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (isnotnull(customer_sk#8) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [3]: [sum(cast(CASE WHEN (isnotnull(customer_sk#8) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint))#23, sum(cast(CASE WHEN (isnull(customer_sk#8) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint))#24, sum(cast(CASE WHEN (isnotnull(customer_sk#8) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint))#25] -Results [3]: [sum(cast(CASE WHEN (isnotnull(customer_sk#8) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint))#23 AS store_only#26, sum(cast(CASE WHEN (isnull(customer_sk#8) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint))#24 AS catalog_only#27, sum(cast(CASE WHEN (isnotnull(customer_sk#8) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END as bigint))#25 AS store_and_catalog#28] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q97/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q97/simplified.txt deleted file mode 100644 index c5921a11c..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q97/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (8) - HashAggregate [sum,sum,sum] [sum(cast(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),store_only,catalog_only,store_and_catalog,sum,sum,sum] - InputAdapter - Exchange #1 - WholeStageCodegen (7) - HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] - Project [customer_sk,customer_sk] - InputAdapter - SortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - WholeStageCodegen (3) - Sort [customer_sk,item_sk] - HashAggregate [ss_customer_sk,ss_item_sk] [customer_sk,item_sk] - InputAdapter - Exchange [ss_customer_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - WholeStageCodegen (6) - Sort [customer_sk,item_sk] - HashAggregate [cs_bill_customer_sk,cs_item_sk] [customer_sk,item_sk] - InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #4 - WholeStageCodegen (5) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q98/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q98/explain.txt deleted file mode 100644 index 11519207d..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q98/explain.txt +++ /dev/null @@ -1,147 +0,0 @@ -== Physical Plan == -* Project (26) -+- * Sort (25) - +- Exchange (24) - +- * Project (23) - +- Window (22) - +- * Sort (21) - +- Exchange (20) - +- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * BroadcastHashJoin Inner BuildRight (15) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.item (4) - +- BroadcastExchange (14) - +- * Project (13) - +- * Filter (12) - +- * ColumnarToRow (11) - +- Scan parquet default.date_dim (10) - - -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) - -(4) Scan parquet default.item -Output [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] - -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Condition : (i_category#9 IN (Sports,Books,Home) AND isnotnull(i_item_sk#4)) - -(7) BroadcastExchange -Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#10] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#4] -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] - -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] - -(12) Filter [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10644)) AND (d_date#12 <= 10674)) AND isnotnull(d_date_sk#11)) - -(13) Project [codegen id : 2] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_date#12] - -(14) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#11] -Join condition: None - -(16) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Input [8]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9, d_date_sk#11] - -(17) HashAggregate [codegen id : 3] -Input [6]: [ss_ext_sales_price#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9] -Keys [5]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#14] -Results [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#15] - -(18) Exchange -Input [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#15] -Arguments: hashpartitioning(i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, 5), true, [id=#16] - -(19) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#15] -Keys [5]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#17] -Results [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS _w0#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS _w1#20, i_item_id#5] - -(20) Exchange -Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5] -Arguments: hashpartitioning(i_class#8, 5), true, [id=#21] - -(21) Sort [codegen id : 5] -Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5] -Arguments: [i_class#8 ASC NULLS FIRST], false, 0 - -(22) Window -Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5] -Arguments: [sum(_w1#20) windowspecdefinition(i_class#8, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#22], [i_class#8] - -(23) Project [codegen id : 6] -Output [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#19) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#22)), DecimalType(38,17), true) AS revenueratio#23, i_item_id#5] -Input [9]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, _w0#19, _w1#20, i_item_id#5, _we0#22] - -(24) Exchange -Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, revenueratio#23, i_item_id#5] -Arguments: rangepartitioning(i_category#9 ASC NULLS FIRST, i_class#8 ASC NULLS FIRST, i_item_id#5 ASC NULLS FIRST, i_item_desc#6 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST, 5), true, [id=#24] - -(25) Sort [codegen id : 7] -Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, revenueratio#23, i_item_id#5] -Arguments: [i_category#9 ASC NULLS FIRST, i_class#8 ASC NULLS FIRST, i_item_id#5 ASC NULLS FIRST, i_item_desc#6 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], true, 0 - -(26) Project [codegen id : 7] -Output [6]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, revenueratio#23] -Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#18, revenueratio#23, i_item_id#5] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q98/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q98/simplified.txt deleted file mode 100644 index 2af712236..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q98/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (7) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] - InputAdapter - Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w1,i_class] - WholeStageCodegen (5) - Sort [i_class] - InputAdapter - Exchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,_w1,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q99/explain.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q99/explain.txt deleted file mode 100644 index 595cb2984..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q99/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.warehouse (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.ship_mode (10) - : +- BroadcastExchange (19) - : +- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet default.call_center (16) - +- BroadcastExchange (26) - +- * Project (25) - +- * Filter (24) - +- * ColumnarToRow (23) - +- Scan parquet default.date_dim (22) - - -(1) Scan parquet default.catalog_sales -Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] - -(3) Filter [codegen id : 5] -Input [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] -Condition : (((isnotnull(cs_warehouse_sk#5) AND isnotnull(cs_ship_mode_sk#4)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_ship_date_sk#2)) - -(4) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) Filter [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(7) BroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_warehouse_sk#5] -Right keys [1]: [w_warehouse_sk#6] -Join condition: None - -(9) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, w_warehouse_name#7] -Input [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5, w_warehouse_sk#6, w_warehouse_name#7] - -(10) Scan parquet default.ship_mode -Output [2]: [sm_ship_mode_sk#9, sm_type#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [sm_ship_mode_sk#9, sm_type#10] - -(12) Filter [codegen id : 2] -Input [2]: [sm_ship_mode_sk#9, sm_type#10] -Condition : isnotnull(sm_ship_mode_sk#9) - -(13) BroadcastExchange -Input [2]: [sm_ship_mode_sk#9, sm_type#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] - -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_ship_mode_sk#4] -Right keys [1]: [sm_ship_mode_sk#9] -Join condition: None - -(15) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, w_warehouse_name#7, sm_type#10] -Input [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, w_warehouse_name#7, sm_ship_mode_sk#9, sm_type#10] - -(16) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#12, cc_name#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 3] -Input [2]: [cc_call_center_sk#12, cc_name#13] - -(18) Filter [codegen id : 3] -Input [2]: [cc_call_center_sk#12, cc_name#13] -Condition : isnotnull(cc_call_center_sk#12) - -(19) BroadcastExchange -Input [2]: [cc_call_center_sk#12, cc_name#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_call_center_sk#3] -Right keys [1]: [cc_call_center_sk#12] -Join condition: None - -(21) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, w_warehouse_name#7, sm_type#10, cc_name#13] -Input [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, w_warehouse_name#7, sm_type#10, cc_call_center_sk#12, cc_name#13] - -(22) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_month_seq#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(23) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#15, d_month_seq#16] - -(24) Filter [codegen id : 4] -Input [2]: [d_date_sk#15, d_month_seq#16] -Condition : (((isnotnull(d_month_seq#16) AND (d_month_seq#16 >= 1200)) AND (d_month_seq#16 <= 1211)) AND isnotnull(d_date_sk#15)) - -(25) Project [codegen id : 4] -Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_month_seq#16] - -(26) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] - -(27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_ship_date_sk#2] -Right keys [1]: [d_date_sk#15] -Join condition: None - -(28) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, w_warehouse_name#7, sm_type#10, cc_name#13] -Input [6]: [cs_sold_date_sk#1, cs_ship_date_sk#2, w_warehouse_name#7, sm_type#10, cc_name#13, d_date_sk#15] - -(29) HashAggregate [codegen id : 5] -Input [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, w_warehouse_name#7, sm_type#10, cc_name#13] -Keys [3]: [substr(w_warehouse_name#7, 1, 20) AS substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, cc_name#13] -Functions [5]: [partial_sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum#19, sum#20, sum#21, sum#22, sum#23] -Results [8]: [substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, cc_name#13, sum#24, sum#25, sum#26, sum#27, sum#28] - -(30) Exchange -Input [8]: [substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, cc_name#13, sum#24, sum#25, sum#26, sum#27, sum#28] -Arguments: hashpartitioning(substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, cc_name#13, 5), true, [id=#29] - -(31) HashAggregate [codegen id : 6] -Input [8]: [substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, cc_name#13, sum#24, sum#25, sum#26, sum#27, sum#28] -Keys [3]: [substr(w_warehouse_name#7, 1, 20)#18, sm_type#10, cc_name#13] -Functions [5]: [sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33, sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34] -Results [8]: [substr(w_warehouse_name#7, 1, 20)#18 AS substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30 AS 30 days #36, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31 AS 31 - 60 days #37, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32 AS 61 - 90 days #38, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33 AS 91 - 120 days #39, sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34 AS >120 days #40] - -(32) TakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] -Arguments: 100, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] - diff --git a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q99/simplified.txt b/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q99/simplified.txt deleted file mode 100644 index 9ebaaac52..000000000 --- a/src/test/resources/tpcds/spark-2.4/approved-plans-v1_4/q99/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) - HashAggregate [substr(w_warehouse_name, 1, 20),sm_type,cc_name,sum,sum,sum,sum,sum] [sum(cast(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END as bigint)),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - InputAdapter - Exchange [substr(w_warehouse_name, 1, 20),sm_type,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [w_warehouse_name,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum] - Project [cs_sold_date_sk,cs_ship_date_sk,w_warehouse_name,sm_type,cc_name] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,w_warehouse_name,sm_type,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,w_warehouse_name,sm_type] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,w_warehouse_name] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Filter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [sm_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet default.ship_mode [sm_ship_mode_sk,sm_type] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [cc_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] diff --git a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala index 6d78215e8..122a6d122 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTest.scala @@ -17,14 +17,17 @@ package com.microsoft.hyperspace.index import scala.collection.mutable.WrappedArray - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.plans.SQLHelper - import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} import com.microsoft.hyperspace.util.FileUtils +import org.apache.commons.codec.binary.Base64 +import org.apache.commons.io.output.ByteArrayOutputStream +import org.apache.spark.util.sketch.BloomFilter + +import java.io.ByteArrayInputStream class CreateIndexTest extends HyperspaceSuite with SQLHelper { override val systemPath = new Path("src/test/resources/indexLocation") @@ -77,6 +80,59 @@ class CreateIndexTest extends HyperspaceSuite with SQLHelper { FileUtils.delete(systemPath) } + test("Sho How Bloom Works") { + // How BF are created + val mbfS = spark.read.parquet(marvelDataPath).select("Affiliation") + .stat.bloomFilter("Affiliation", 100, 0.03) + + val dbfS = spark.read.parquet(dcDataPath).select("Affiliation") + .stat.bloomFilter("Affiliation", 100, 0.03) + + // All BF are stored locally and read again + val mbfByteStream = new ByteArrayOutputStream() + mbfS.writeTo(mbfByteStream) + + val dbfByteStream = new ByteArrayOutputStream() + dbfS.writeTo(dbfByteStream) + val bloomDF = spark.createDataFrame( + Seq( + ("marvel", Base64.encodeBase64String(mbfByteStream.toByteArray)), + ("dc", Base64.encodeBase64String(dbfByteStream.toByteArray)) + ) + ).toDF("FileName", "Data") + bloomDF.write.parquet(new Path(comicDataDir, "bf.parquet").toString) + + // BF read + val bloomDFR = spark.read.parquet(new Path(comicDataDir, "bf.parquet").toString) + + // BF used to check for filters +// bloomDFR.filter().select("filename") + bloomDFR.filter("FileName = 'marvel'").select("Data").collect() + .foreach { + row => { + val mbf = BloomFilter.readFrom(new ByteArrayInputStream( + Base64.decodeBase64(row.getString(0)))) + assert(mbf.mightContain("XMen")) + assert(mbf.mightContainString("Avengers")) + assert(!mbf.mightContainString("Suicide Squad")) + assert(!mbf.mightContainString("Justice League")) + assert(!mbf.mightContainString("Suicide")) + } + } + bloomDFR.filter("FileName = 'dc'").select("Data").collect() + .foreach{ + row => { + val dbf = BloomFilter.readFrom(new ByteArrayInputStream( + Base64.decodeBase64(row.getString(0)))) + assert(!dbf.mightContainString("XMen")) + assert(!dbf.mightContainString("Avengers")) + assert(dbf.mightContainString("Suicide Squad")) + assert(dbf.mightContainString("Justice League")) + assert(!dbf.mightContainString("Justice")) + } + } + } + test("Creating one covering index.") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig1) val count = hyperspace.indexes.where(s"name = '${indexConfig1.indexName}' ").count diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala index 2c8e59e70..1eb596915 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.types.{HIVE_TYPE_STRING, IntegerType, StringType, StructType} + import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.util.PathUtils diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala index 9443a8b46..0e8a04109 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala @@ -106,7 +106,7 @@ trait HyperspaceRuleSuite extends HyperspaceSuite { null, LogicalPlanFingerprint(LogicalPlanFingerprint.Properties(Seq(Signature(signClass, s))))) - val indexFiles = getIndexDataFilesPaths(name).map { path => + val indexFiles = getBloomDataFilePath(name).map { path => new FileStatus(10, false, 1, 10, 10, path) } @@ -133,8 +133,8 @@ trait HyperspaceRuleSuite extends HyperspaceSuite { } } - def getBloomDataFilePath(indexName: String): Option[Path] = { - Some( + def getBloomDataFilePath(indexName: String): Seq[Path] = { + Seq( new Path ( new Path ( new Path (systemPath, indexName),