diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 317303eb7b..a82a33151f 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -307,6 +307,18 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) + val COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.native.shuffle.partitioning.hash.enabled") + .doc("Whether to enable hash partitioning for Comet native shuffle.") + .booleanConf + .createWithDefault(true) + + val COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.native.shuffle.partitioning.range.enabled") + .doc("Whether to enable range partitioning for Comet native shuffle.") + .booleanConf + .createWithDefault(true) + val COMET_EXEC_SHUFFLE_COMPRESSION_CODEC: ConfigEntry[String] = conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec") .doc( @@ -770,11 +782,13 @@ private[comet] abstract class ConfigEntry[T]( /** * Retrieves the config value from the current thread-local [[SQLConf]] + * * @return */ def get(): T = get(SQLConf.get) def defaultValue: Option[T] = None + def defaultValueString: String override def toString: String = { @@ -793,6 +807,7 @@ private[comet] class ConfigEntryWithDefault[T]( version: String) extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic, version) { override def defaultValue: Option[T] = Some(_defaultValue) + override def defaultValueString: String = stringConverter(_defaultValue) def get(conf: SQLConf): T = { @@ -828,6 +843,7 @@ private[comet] class OptionalConfigEntry[T]( } private[comet] case class ConfigBuilder(key: String) { + import ConfigHelpers._ var _public = true diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 52aa73e93d..5233cdb000 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -2282,10 +2282,17 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 266bb343526..c3e3d155813 100644 +index 266bb343526..6675cf7b636 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -@@ -24,10 +24,11 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec +@@ -19,15 +19,18 @@ package org.apache.spark.sql.sources + + import scala.util.Random + ++import org.apache.comet.CometConf ++ + import org.apache.spark.sql._ + import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning @@ -2299,7 +2306,7 @@ index 266bb343526..c3e3d155813 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -101,12 +102,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -101,12 +104,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -2322,7 +2329,7 @@ index 266bb343526..c3e3d155813 100644 // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -155,7 +164,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -155,7 +166,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -2332,7 +2339,7 @@ index 266bb343526..c3e3d155813 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -451,28 +461,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -451,28 +463,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2390,7 +2397,7 @@ index 266bb343526..c3e3d155813 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -835,11 +866,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -835,11 +868,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2404,7 +2411,40 @@ index 266bb343526..c3e3d155813 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1026,15 +1057,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -894,7 +927,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + + test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { + val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) +@@ -913,7 +949,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + + test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "9", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { + +@@ -943,7 +982,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + + test("bucket coalescing eliminates shuffle") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. +@@ -1026,15 +1068,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti expectedNumShuffles: Int, expectedCoalescedNumBuckets: Option[Int]): Unit = { val plan = sql(query).queryExecution.executedPlan @@ -2816,6 +2856,34 @@ index 52abd248f3a..7a199931a08 100644 case h: HiveTableScanExec => h.partitionPruningPred.collect { case d: DynamicPruningExpression => d.child } +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +index a902cb3a69e..800a3acbe99 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +@@ -24,6 +24,7 @@ import java.sql.{Date, Timestamp} + import java.util.{Locale, Set} + + import com.google.common.io.Files ++import org.apache.comet.CometConf + import org.apache.hadoop.fs.{FileSystem, Path} + + import org.apache.spark.{SparkException, TestUtils} +@@ -838,8 +839,13 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi + } + + test("SPARK-2554 SumDistinct partial aggregation") { +- checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), +- sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. ++ withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false") ++ { ++ checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), ++ sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ } + } + + test("SPARK-4963 DataFrame sample on mutable row return wrong result") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 07361cfdce9..b4d53dbe900 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala diff --git a/dev/diffs/3.5.6.diff b/dev/diffs/3.5.6.diff index 4897de62a7..0d35cbf7ca 100644 --- a/dev/diffs/3.5.6.diff +++ b/dev/diffs/3.5.6.diff @@ -2278,10 +2278,18 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 746f289c393..0c99d028163 100644 +index 746f289c393..a90106a1463 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -@@ -25,10 +25,11 @@ import org.apache.spark.sql.catalyst.expressions +@@ -19,16 +19,19 @@ package org.apache.spark.sql.sources + + import scala.util.Random + ++import org.apache.comet.CometConf ++ + import org.apache.spark.sql._ + import org.apache.spark.sql.catalyst.catalog.BucketSpec + import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -2295,7 +2303,7 @@ index 746f289c393..0c99d028163 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -102,12 +103,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -102,12 +105,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -2318,7 +2326,7 @@ index 746f289c393..0c99d028163 100644 // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -156,7 +165,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -156,7 +167,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -2328,7 +2336,7 @@ index 746f289c393..0c99d028163 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -452,28 +462,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -452,28 +464,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2386,7 +2394,7 @@ index 746f289c393..0c99d028163 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -836,11 +867,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -836,11 +869,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2400,7 +2408,40 @@ index 746f289c393..0c99d028163 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1029,15 +1060,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -895,7 +928,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + + test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { + val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) +@@ -914,7 +950,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + + test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "9", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { + +@@ -944,7 +983,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + + test("bucket coalescing eliminates shuffle") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. +@@ -1029,15 +1071,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -2830,6 +2871,34 @@ index 549431ef4f4..e48f1730da6 100644 withTempDir { dir => withSQLConf( "parquet.crypto.factory.class" -> +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +index 6160c3e5f6c..0956d7d9edc 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +@@ -24,6 +24,7 @@ import java.sql.{Date, Timestamp} + import java.util.{Locale, Set} + + import com.google.common.io.Files ++import org.apache.comet.CometConf + import org.apache.hadoop.fs.{FileSystem, Path} + + import org.apache.spark.{SparkException, TestUtils} +@@ -838,8 +839,13 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi + } + + test("SPARK-2554 SumDistinct partial aggregation") { +- checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), +- sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. ++ withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false") ++ { ++ checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), ++ sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ } + } + + test("SPARK-4963 DataFrame sample on mutable row return wrong result") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 1d646f40b3e..7f2cdb8f061 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index f4cb601e41..f5d9197481 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -595,7 +595,7 @@ index 49a33d1c925..197c93d62b3 100644 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala new file mode 100644 -index 00000000000..5db1532cf9f +index 00000000000..5691536c114 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala @@ -0,0 +1,45 @@ @@ -2517,10 +2517,18 @@ index 6ff07449c0c..9f95cff99e5 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 3573bafe482..11d387110ea 100644 +index 3573bafe482..8bf48a94fba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -@@ -25,10 +25,11 @@ import org.apache.spark.sql.catalyst.expressions +@@ -19,16 +19,19 @@ package org.apache.spark.sql.sources + + import scala.util.Random + ++import org.apache.comet.CometConf ++ + import org.apache.spark.sql._ + import org.apache.spark.sql.catalyst.catalog.BucketSpec + import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -2534,7 +2542,7 @@ index 3573bafe482..11d387110ea 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -102,12 +103,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -102,12 +105,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -2557,7 +2565,7 @@ index 3573bafe482..11d387110ea 100644 // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -156,7 +165,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -156,7 +167,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -2567,7 +2575,7 @@ index 3573bafe482..11d387110ea 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -452,28 +462,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -452,28 +464,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sparkSession.sessionState.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2625,7 +2633,7 @@ index 3573bafe482..11d387110ea 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -836,11 +867,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -836,11 +869,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2639,7 +2647,40 @@ index 3573bafe482..11d387110ea 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1029,15 +1060,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -895,7 +928,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + + test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { + val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) +@@ -914,7 +950,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + + test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "9", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { + +@@ -944,7 +983,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + + test("bucket coalescing eliminates shuffle") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. +@@ -1029,15 +1071,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -3036,6 +3077,34 @@ index 52abd248f3a..7a199931a08 100644 case h: HiveTableScanExec => h.partitionPruningPred.collect { case d: DynamicPruningExpression => d.child } +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +index 0bcac639443..8957c76886f 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +@@ -24,6 +24,7 @@ import java.sql.{Date, Timestamp} + import java.util.{Locale, Set} + + import com.google.common.io.Files ++import org.apache.comet.CometConf + import org.apache.hadoop.fs.{FileSystem, Path} + + import org.apache.spark.{SparkException, TestUtils} +@@ -838,8 +839,13 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi + } + + test("SPARK-2554 SumDistinct partial aggregation") { +- checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), +- sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. ++ withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false") ++ { ++ checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), ++ sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ } + } + + test("SPARK-4963 DataFrame sample on mutable row return wrong result") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 3f8de93b330..53417076481 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 517ce960c0..72a3c03548 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -75,6 +75,8 @@ Comet provides the following configuration settings. | spark.comet.memory.overhead.min | Minimum amount of additional memory to be allocated per executor process for Comet, in MiB, when running Spark in on-heap mode. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | 402653184b | | spark.comet.memoryOverhead | The amount of additional memory to be allocated per executor process for Comet, in MiB, when running Spark in on-heap mode. This config is optional. If this is not specified, it will be set to `spark.comet.memory.overhead.factor` * `spark.executor.memory`. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | | | spark.comet.metrics.updateInterval | The interval in milliseconds to update metrics. If interval is negative, metrics will be updated upon task completion. | 3000 | +| spark.comet.native.shuffle.partitioning.hash.enabled | Whether to enable hash partitioning for Comet native shuffle. | true | +| spark.comet.native.shuffle.partitioning.range.enabled | Whether to enable range partitioning for Comet native shuffle. | true | | spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false | | spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. | false | | spark.comet.parquet.read.io.adjust.readRange.skew | In the parallel reader, if the read ranges submitted are skewed in sizes, this option will cause the reader to break up larger read ranges into smaller ranges to reduce the skew. This will result in a slightly larger number of connections opened to the file system but may give improved performance. | false | diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 0fd7502351..ee07462d00 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -18,11 +18,14 @@ use arrow::array::builder::{Date32Builder, Decimal128Builder, Int32Builder}; use arrow::array::{builder::StringBuilder, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; -use comet::execution::shuffle::{CompressionCodec, ShuffleBlockWriter, ShuffleWriterExec}; +use comet::execution::shuffle::{ + CometPartitioning, CompressionCodec, ShuffleBlockWriter, ShuffleWriterExec, +}; use criterion::{criterion_group, criterion_main, Criterion}; use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::source::DataSourceExec; -use datafusion::physical_expr::{expressions::Column, Partitioning}; +use datafusion::physical_expr::expressions::{col, Column}; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion::physical_plan::metrics::Time; use datafusion::{ physical_plan::{common::collect, ExecutionPlan}, @@ -42,20 +45,18 @@ fn criterion_benchmark(c: &mut Criterion) { CompressionCodec::Zstd(1), CompressionCodec::Zstd(6), ] { - for enable_fast_encoding in [true, false] { - let name = format!("shuffle_writer: write encoded (enable_fast_encoding={enable_fast_encoding}, compression={compression_codec:?})"); - group.bench_function(name, |b| { - let mut buffer = vec![]; - let ipc_time = Time::default(); - let w = ShuffleBlockWriter::try_new(&batch.schema(), compression_codec.clone()) - .unwrap(); - b.iter(|| { - buffer.clear(); - let mut cursor = Cursor::new(&mut buffer); - w.write_batch(&batch, &mut cursor, &ipc_time).unwrap(); - }); + let name = format!("shuffle_writer: write encoded (compression={compression_codec:?})"); + group.bench_function(name, |b| { + let mut buffer = vec![]; + let ipc_time = Time::default(); + let w = + ShuffleBlockWriter::try_new(&batch.schema(), compression_codec.clone()).unwrap(); + b.iter(|| { + buffer.clear(); + let mut cursor = Cursor::new(&mut buffer); + w.write_batch(&batch, &mut cursor, &ipc_time).unwrap(); }); - } + }); } for compression_codec in [ @@ -66,10 +67,40 @@ fn criterion_benchmark(c: &mut Criterion) { CompressionCodec::Zstd(6), ] { group.bench_function( - format!("shuffle_writer: end to end (compression = {compression_codec:?}"), + format!("shuffle_writer: end to end (compression = {compression_codec:?})"), + |b| { + let ctx = SessionContext::new(); + let exec = create_shuffle_writer_exec( + compression_codec.clone(), + CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), + ); + b.iter(|| { + let task_ctx = ctx.task_ctx(); + let stream = exec.execute(0, task_ctx).unwrap(); + let rt = Runtime::new().unwrap(); + rt.block_on(collect(stream)).unwrap(); + }); + }, + ); + } + + for partitioning in [ + CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), + CometPartitioning::RangePartitioning( + LexOrdering::new(vec![PhysicalSortExpr::new_default( + col("c0", batch.schema().as_ref()).unwrap(), + )]), + 16, + 100, + ), + ] { + let compression_codec = CompressionCodec::None; + group.bench_function( + format!("shuffle_writer: end to end (partitioning={partitioning:?})"), |b| { let ctx = SessionContext::new(); - let exec = create_shuffle_writer_exec(compression_codec.clone()); + let exec = + create_shuffle_writer_exec(compression_codec.clone(), partitioning.clone()); b.iter(|| { let task_ctx = ctx.task_ctx(); let stream = exec.execute(0, task_ctx).unwrap(); @@ -81,7 +112,10 @@ fn criterion_benchmark(c: &mut Criterion) { } } -fn create_shuffle_writer_exec(compression_codec: CompressionCodec) -> ShuffleWriterExec { +fn create_shuffle_writer_exec( + compression_codec: CompressionCodec, + partitioning: CometPartitioning, +) -> ShuffleWriterExec { let batches = create_batches(8192, 10); let schema = batches[0].schema(); let partitions = &[batches]; @@ -89,7 +123,7 @@ fn create_shuffle_writer_exec(compression_codec: CompressionCodec) -> ShuffleWri Arc::new(DataSourceExec::new(Arc::new( MemorySourceConfig::try_new(partitions, Arc::clone(&schema), None).unwrap(), ))), - Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), + partitioning, compression_codec, "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 0eb3d5cc29..511c09f3ec 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -60,7 +60,7 @@ use datafusion::{ limit::LocalLimitExec, projection::ProjectionExec, sorts::sort::SortExec, - ExecutionPlan, Partitioning, + ExecutionPlan, }, prelude::SessionContext, }; @@ -69,7 +69,7 @@ use datafusion_comet_spark_expr::{ }; use crate::execution::operators::ExecutionError::GeneralError; -use crate::execution::shuffle::CompressionCodec; +use crate::execution::shuffle::{CometPartitioning, CompressionCodec}; use crate::execution::spark_plan::SparkPlan; use crate::parquet::parquet_support::prepare_object_store_with_configs; use datafusion::common::scalar::ScalarStructBuilder; @@ -921,7 +921,7 @@ impl PhysicalPlanner { let children = &spark_plan.children; match spark_plan.op_struct.as_ref().unwrap() { OpStruct::Projection(project) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let exprs: PhyExprResult = project .project_list @@ -942,7 +942,7 @@ impl PhysicalPlanner { )) } OpStruct::Filter(filter) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let predicate = self.create_expr(filter.predicate.as_ref().unwrap(), child.schema())?; @@ -965,7 +965,7 @@ impl PhysicalPlanner { )) } OpStruct::HashAgg(agg) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let group_exprs: PhyExprResult = agg @@ -1043,7 +1043,7 @@ impl PhysicalPlanner { } } OpStruct::Limit(limit) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let num = limit.limit; let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; @@ -1057,7 +1057,7 @@ impl PhysicalPlanner { )) } OpStruct::Sort(sort) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let exprs: Result, ExecutionError> = sort @@ -1225,7 +1225,7 @@ impl PhysicalPlanner { )) } OpStruct::ShuffleWriter(writer) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let partitioning = self @@ -1263,7 +1263,7 @@ impl PhysicalPlanner { )) } OpStruct::Expand(expand) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let mut projections = vec![]; @@ -1509,7 +1509,7 @@ impl PhysicalPlanner { condition: &Option, partition_count: usize, ) -> Result<(JoinParameters, Vec), ExecutionError> { - assert!(children.len() == 2); + assert_eq!(children.len(), 2); let (mut left_scans, left) = self.create_plan(&children[0], inputs, partition_count)?; let (mut right_scans, right) = self.create_plan(&children[1], inputs, partition_count)?; @@ -2189,7 +2189,7 @@ impl PhysicalPlanner { &self, spark_partitioning: &SparkPartitioning, input_schema: SchemaRef, - ) -> Result { + ) -> Result { match spark_partitioning.partitioning_struct.as_ref().unwrap() { PartitioningStruct::HashPartition(hash_partition) => { let exprs: PartitionPhyExprResult = hash_partition @@ -2197,12 +2197,25 @@ impl PhysicalPlanner { .iter() .map(|x| self.create_expr(x, Arc::clone(&input_schema))) .collect(); - Ok(Partitioning::Hash( + Ok(CometPartitioning::Hash( exprs?, hash_partition.num_partitions as usize, )) } - PartitioningStruct::SinglePartition(_) => Ok(Partitioning::UnknownPartitioning(1)), + PartitioningStruct::RangePartition(range_partition) => { + let exprs: Result, ExecutionError> = range_partition + .sort_orders + .iter() + .map(|expr| self.create_sort_expr(expr, Arc::clone(&input_schema))) + .collect(); + let lex_ordering = LexOrdering::from(exprs?); + Ok(CometPartitioning::RangePartitioning( + lex_ordering, + range_partition.num_partitions as usize, + range_partition.sample_size as usize, + )) + } + PartitioningStruct::SinglePartition(_) => Ok(CometPartitioning::SinglePartition), } } diff --git a/native/core/src/execution/shuffle/comet_partitioning.rs b/native/core/src/execution/shuffle/comet_partitioning.rs new file mode 100644 index 0000000000..b063473bdf --- /dev/null +++ b/native/core/src/execution/shuffle/comet_partitioning.rs @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use datafusion::physical_expr::{LexOrdering, PhysicalExpr}; +use std::sync::Arc; + +#[derive(Debug, Clone)] +pub enum CometPartitioning { + SinglePartition, + /// Allocate rows based on a hash of one of more expressions and the specified number of + /// partitions + Hash(Vec>, usize), + /// Allocate rows based on lexical order of one of more expressions and the specified number of + /// partitions + RangePartitioning(LexOrdering, usize, usize), +} + +impl CometPartitioning { + pub fn partition_count(&self) -> usize { + use CometPartitioning::*; + match self { + SinglePartition => 1, + Hash(_, n) | RangePartitioning(_, n, _) => *n, + } + } +} diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index 4864ffc062..c3cb610afa 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -16,10 +16,13 @@ // under the License. pub(crate) mod codec; +mod comet_partitioning; mod list; mod map; +mod range_partitioner; pub mod row; mod shuffle_writer; pub use codec::{read_ipc_compressed, CompressionCodec, ShuffleBlockWriter}; +pub use comet_partitioning::CometPartitioning; pub use shuffle_writer::ShuffleWriterExec; diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs new file mode 100644 index 0000000000..8f443232f9 --- /dev/null +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -0,0 +1,474 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::{ArrayRef, UInt64Array}; +use arrow::compute::{take_arrays, TakeOptions}; +use arrow::row::{OwnedRow, Row, RowConverter, Rows, SortField}; +use datafusion::error::{DataFusionError, Result}; +use datafusion::physical_expr::LexOrdering; +use rand::{rngs::SmallRng, Rng, SeedableRng}; + +pub struct RangePartitioner; + +impl RangePartitioner { + /// Given a number of rows, sample size, and a random seed, generates unique indices to take + /// from an input batch to act as a random sample. + /// Adapted from https://en.wikipedia.org/wiki/Reservoir_sampling#Optimal:_Algorithm_L + /// We use sample_size instead of k and num_rows instead of n. + /// We use indices instead of actual values in the reservoir since we'll do one take() on the + /// input arrays at the end. + fn reservoir_sample_indices(num_rows: usize, sample_size: usize, seed: u64) -> Vec { + assert!(sample_size > 0); + assert!( + num_rows > sample_size, + "Sample size > num_rows yields original batch." + ); + + // Initialize our reservoir with indices of the first |sample_size| elements. + let mut reservoir: Vec = (0..sample_size as u64).collect(); + + let mut rng = SmallRng::seed_from_u64(seed); + let mut w = (rng.random::().ln() / sample_size as f64).exp(); + let mut i = sample_size - 1; + + while i < num_rows { + i += (rng.random::().ln() / (1.0 - w).ln()).floor() as usize + 1; + + if i < num_rows { + // Replace a random item in the reservoir with i + let random_index = rng.random_range(0..sample_size); + reservoir[random_index] = i as u64; + w *= (rng.random::().ln() / sample_size as f64).exp(); + } + } + + reservoir + } + + /// Given a batch of Rows, an ordered vector of Rows that represent partition boundaries, and + /// a slice with enough space for the input batch, determines a partition id for every input + /// Row using binary search. + pub fn partition_indices_for_batch( + row_batch: &Rows, + partition_bounds_vec: &[OwnedRow], + partition_ids: &mut [u32], + ) { + row_batch.iter().enumerate().for_each(|(row_idx, row)| { + partition_ids[row_idx] = + partition_bounds_vec.partition_point(|bound| bound.row() <= row) as u32 + }); + } + + /// Given input arrays and range partitioning metadata: samples the input arrays, generates + /// partition bounds, and returns Rows (for comparison against) and a RowConverter (for + /// adapting future incoming batches). + pub fn generate_bounds( + partition_arrays: &Vec, + lex_ordering: &LexOrdering, + num_output_partitions: usize, + num_rows: usize, + sample_size: usize, + seed: u64, + ) -> Result<(Rows, RowConverter), DataFusionError> { + let sampled_columns = if sample_size < num_rows { + // Construct our sample indices. + let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( + num_rows, + sample_size, + seed, + )); + + // Extract our sampled data from the input data. + take_arrays( + partition_arrays, + &sample_indices, + Some(TakeOptions { + check_bounds: false, + }), + )? + } else { + // Requested sample_size is larger than the batch, so just use the batch. + partition_arrays.clone() + }; + + // Generate our bounds indices. + let sort_fields: Vec = partition_arrays + .iter() + .zip(lex_ordering) + .map(|(array, sort_expr)| { + SortField::new_with_options(array.data_type().clone(), sort_expr.options) + }) + .collect(); + + let (bounds_indices, row_converter) = RangePartitioner::determine_bounds_for_rows( + sort_fields, + sampled_columns.as_slice(), + num_output_partitions, + )?; + + // Extract our bounds data from the sampled data. + let bounds_indices_array = UInt64Array::from(bounds_indices); + let bounds_arrays = take_arrays( + sampled_columns.as_slice(), + &bounds_indices_array, + Some(TakeOptions { + check_bounds: false, + }), + )?; + + // Convert the bounds data to Rows and return with RowConverter. + Ok(( + row_converter.convert_columns(bounds_arrays.as_slice())?, + row_converter, + )) + } + + /// Given a sort ordering, sampled data, and a number of target partitions, finds the partition + /// bounds and returns them as indices into the sampled data. + /// Adapted from org.apache.spark.RangePartitioner.determineBounds but without weighted + /// values since we don't have cross-partition samples to merge. + /// We normalize the math to use ints instead of floating point by replacing 1.0 with a + /// (imagined) num_candidates * partitions range. + fn determine_bounds_for_rows( + sort_fields: Vec, + sampled_columns: &[ArrayRef], + partitions: usize, + ) -> Result<(Vec, RowConverter), DataFusionError> { + assert!(partitions > 1); + + let converter = RowConverter::new(sort_fields)?; + let sampled_rows = converter.convert_columns(sampled_columns)?; + let mut sorted_sampled_rows: Vec<(usize, Row)> = sampled_rows.iter().enumerate().collect(); + sorted_sampled_rows.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); + + let num_candidates = sampled_rows.num_rows(); + let step = 1.0 / partitions as f64; + let mut cumulative_weights = 0.0; + let mut target = step; + let mut bounds_indices: Vec = Vec::with_capacity(partitions - 1); + let mut i = 0; + let mut j = 0; + let mut previous_bound = None; + let sample_weight = 1.0 / num_candidates as f64; + while (i < num_candidates) && (j < partitions - 1) { + let key = sorted_sampled_rows[i]; + cumulative_weights += sample_weight; + if cumulative_weights >= target { + // Skip duplicate values. + if previous_bound.is_none() || key.1 > previous_bound.unwrap() { + bounds_indices.push(key.0 as u64); + target += step; + j += 1; + previous_bound = Some(key.1) + } + } + i += 1 + } + + Ok((bounds_indices, converter)) + } +} + +#[cfg(test)] +mod test { + use super::*; + use arrow::array::{Array, AsArray, Int64Array, RecordBatch, UInt64Array}; + use arrow::compute::take_record_batch; + use arrow::datatypes::DataType::{Float64, Int64}; + use arrow::datatypes::{Field, Float64Type, Int32Type, Int64Type, Schema}; + use datafusion::common::record_batch; + use datafusion::physical_expr::expressions::col; + use datafusion::physical_expr::PhysicalSortExpr; + use itertools::Itertools; + use std::sync::Arc; + + fn sample_batch(input: RecordBatch, indices: Vec) -> RecordBatch { + let indices = UInt64Array::from(indices); + take_record_batch(&input, &indices).unwrap() + } + + fn check_sample_indices(indices: &[u64], batch_size: usize, sample_size: usize) { + // sample indices size should never exceed the batch size + assert!(indices.len() <= batch_size); + // number of samples should be the smaller of batch size and sample size + assert_eq!(indices.len(), batch_size.min(sample_size)); + // Check that indices are not out of bounds + indices + .iter() + .for_each(|&idx| assert!(idx < batch_size as u64)); + // Check that values are distinct + let sorted_indices = indices.iter().sorted().collect_vec(); + assert_eq!( + sorted_indices.len(), + sorted_indices.iter().dedup().collect_vec().len() + ); + } + + fn check_bounds_indices(indices: &[u64], sample_size: usize) { + // bounds indices size should never exceed the sample size + assert!(indices.len() <= sample_size); + // Check that indices are not out of bounds + indices + .iter() + .for_each(|&idx| assert!(idx < sample_size as u64)); + // Check that values are distinct + let sorted_indices = indices.iter().sorted().collect_vec(); + assert_eq!( + sorted_indices.len(), + sorted_indices.iter().dedup().collect_vec().len() + ); + } + + #[test] + // Reservoir sampling from a sorted batch can appear like an adversarial case, so we want to + // ensure that we still generate reasonable bounds. A sorted batch with partitioning column of + // [0,8192) is sampled and we generate bounds for 10 partitions. A perfect split would have + // ~819 between each partition, so we just assert that the difference in bounds is <1000. + fn generate_bounds_for_sorted_batch() { + let input_batch = create_random_batch(8192, true, Some((0, 8192))); + + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( + col("a", input_batch.schema().as_ref()).unwrap(), + )]); + + let (rows, row_converter) = RangePartitioner::generate_bounds( + input_batch.columns().to_vec().as_ref(), + &lex_ordering, + 10, + input_batch.num_rows(), + 1000, + 42, + ) + .unwrap(); + + let rows_array = row_converter.convert_rows(&rows).unwrap(); + + let primitive_array = rows_array[0].as_primitive::(); + + for i in 0..8 { + assert!( + (primitive_array.values().get(i + 1).unwrap() + - primitive_array.values().get(i).unwrap()) + .abs() + < 1000 + ); + } + } + + #[test] + // We want to verify that with hand-written bounds for a distribution of data that we get + // reasonable partition indices for a random batch. For this scenario, we create a full + // RecordBatch with one partition column. The values in the column are uniform randomly + // distributed between [0,10). We request 10 partitions with bounds of [1,2,3,4,5,6,7,8,9], + // and the result should be 10 bins with reasonably close counts. + fn partition_indices_for_batch() { + let sort_fields = vec![SortField::new(Int64)]; + let row_converter = RowConverter::new(sort_fields).unwrap(); + let mut partition_ids = vec![0u32; 8192]; + let mut partition_counts = [0u32; 10]; + + let input_batch = create_random_batch(8192, false, Some((0, 10))); + let bounds = record_batch!(("a", Int64, (1..=9).collect_vec())).unwrap(); + + let input_rows = row_converter + .convert_columns(input_batch.columns()) + .unwrap(); + + let bounds_rows = row_converter.convert_columns(bounds.columns()).unwrap(); + + let owned_rows_vec = bounds_rows.iter().map(|row| row.owned()).collect_vec(); + + RangePartitioner::partition_indices_for_batch( + &input_rows, + owned_rows_vec.as_slice(), + &mut partition_ids, + ); + + partition_ids + .iter() + .for_each(|&partition_id| partition_counts[partition_id as usize] += 1); + + // The RecordBatch won't be perfectly distributed, so for 8192 / 10 possible values we + // check that each partition just has >700 values in them. + partition_counts + .iter() + .for_each(|&partition_count| assert!(partition_count > 700)); + } + + #[test] + // We want to verify that reservoir sampling yields valid indices for different size input + // batches. We randomly generate batch sizes and sample sizes, and then construct reservoir + // samples for each scenario. Finally, we validate the indices. + fn reservoir_sample_random() { + let mut rng = SmallRng::seed_from_u64(42); + + // These functions don't rely on unsafe Rust, so we reduce the number iterations for CI + // when testing with Miri. + for _ in 0..if cfg!(miri) { 8 } else { 8192 } { + let batch_size: usize = rng.random_range(1..=8192); + // We don't test sample size > batch_size since in that case you would just take the + // entire batch as the sample. + let sample_size: usize = rng.random_range(1..batch_size); + let indices = RangePartitioner::reservoir_sample_indices(batch_size, sample_size, 42); + + check_sample_indices(&indices, batch_size, sample_size); + } + } + + #[test] + // org.apache.spark.util.random.SamplingUtilsSuite + // "SPARK-18678 reservoirSampleAndCount with tiny input" + fn reservoir_sample_and_count_with_tiny_input() { + let batch = record_batch!(("a", Int32, vec![0, 1])).unwrap(); + let mut counts: Vec = vec![0; 2]; + for i in 0..500 { + let indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 1, i); + let result = sample_batch(batch.clone(), indices); + assert_eq!(result.num_rows(), 1); + counts[result.column(0).as_primitive::().value(0) as usize] += 1; + } + // If correct, should be true with prob ~ 0.99999707 according to original Spark test. + assert!((counts[0] - counts[1]).abs() <= 100) + } + + #[test] + // org.apache.spark.PartitioningSuite + // "RangePartitioner.determineBounds" + fn determine_bounds_for_rows() { + // The original test had weights on the values. We just duplicate them because our + // determine_bounds function is unweighted. + let batch = record_batch!(( + "a", + Float64, + vec![ + Some(0.7), + Some(0.7), + Some(0.1), + Some(0.4), + Some(0.3), + Some(0.2), + Some(0.5), + Some(1.0), + Some(1.0), + Some(1.0), + ] + )) + .unwrap(); + + let sort_fields = vec![SortField::new(Float64)]; + + let (rows, _) = + RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 3).unwrap(); + + check_bounds_indices(rows.as_slice(), batch.num_rows()); + + assert_eq!(rows.len(), 2); + + let indices = UInt64Array::from(rows); + + let bounds = take_record_batch(&batch, &indices).unwrap(); + let bounds_array = bounds.column(0).as_primitive::(); + assert_eq!(bounds_array.values(), &[0.4, 0.7]); + } + + #[test] + // We want to verify that determining bounds yields valid indices for different size sample + // batches. We randomly generate batches and number of partitions, and then construct + // bounds for each scenario. Finally, we validate the indices. + fn determine_bounds_random() { + let mut rng = SmallRng::seed_from_u64(42); + + let sort_fields = vec![SortField::new(Int64)]; + + // These functions don't rely on unsafe Rust, so we reduce the number iterations for CI + // when testing with Miri. + for _ in 0..if cfg!(miri) { 8 } else { 2048 } { + let batch_size = rng.random_range(0..=8192); + // We don't test fewer than 2 partitions since this is used by the + // MultiPartitionShuffleRepartitioner which is for >1 partitions. + let num_partitions = rng.random_range(2..=10000); + + let batch = create_random_batch(batch_size, false, None); + + let (rows, _) = RangePartitioner::determine_bounds_for_rows( + sort_fields.clone(), + batch.columns(), + num_partitions, + ) + .unwrap(); + + check_bounds_indices(rows.as_slice(), batch_size as usize); + + let rows_array = UInt64Array::from(rows); + + let bounds = take_record_batch(&batch, &rows_array).unwrap(); + + let bounds_vec: Vec = bounds + .column(0) + .as_primitive::() + .values() + .to_vec(); + + // Bounds should be sorted. + assert!(bounds_vec.is_sorted()); + // Bounds should be unique. + assert_eq!( + bounds_vec.len(), + bounds_vec.iter().dedup().collect_vec().len() + ); + } + } + + #[test] + // We want to make sure that finding bounds works with nulls. DF has more exhaustive tests for + // sorting with nulls, so we defer to those for more coverage. This is just a small + // deterministic test to verify that nulls can be partition boundaries. + fn determine_bounds_with_nulls() { + let batch = record_batch!(("a", Float64, vec![None, None, Some(0.1),])).unwrap(); + + let sort_fields = vec![SortField::new(Float64)]; + + let (rows, _) = + RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 2).unwrap(); + + assert_eq!(rows.len(), 1); + + let indices = UInt64Array::from(rows); + + let bounds = take_record_batch(&batch, &indices).unwrap(); + let bounds_array = bounds.column(0).as_primitive::(); + assert!(bounds_array.is_null(0)); + } + + fn create_random_batch(batch_size: u32, sort: bool, range: Option<(i64, i64)>) -> RecordBatch { + let mut rng = SmallRng::seed_from_u64(42); + let mut column: Vec = if let Some((min, max)) = range { + assert!(min <= max); + (0..batch_size) + .map(|_| rng.random_range(min..max)) + .collect() + } else { + (0..batch_size).map(|_| rng.random::()).collect() + }; + if sort { + column.sort(); + } + let array = Int64Array::from(column); + let schema = Arc::new(Schema::new(vec![Field::new("a", Int64, true)])); + RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap() + } +} diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 16e5516f48..eb3083bff8 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -17,12 +17,14 @@ //! Defines the External shuffle repartition plan. -use crate::execution::shuffle::{CompressionCodec, ShuffleBlockWriter}; +use crate::execution::shuffle::range_partitioner::RangePartitioner; +use crate::execution::shuffle::{CometPartitioning, CompressionCodec, ShuffleBlockWriter}; use crate::execution::tracing::{with_trace, with_trace_async}; use arrow::compute::interleave_record_batch; +use arrow::row::{OwnedRow, RowConverter}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; -use datafusion::physical_expr::EquivalenceProperties; +use datafusion::physical_expr::{EquivalenceProperties, Partitioning}; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::EmptyRecordBatchStream; use datafusion::{ @@ -39,8 +41,8 @@ use datafusion::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, Time, }, stream::RecordBatchStreamAdapter, - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, - SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SendableRecordBatchStream, + Statistics, }, }; use datafusion_comet_spark_expr::hash_funcs::murmur3::create_murmur3_hashes; @@ -66,7 +68,7 @@ pub struct ShuffleWriterExec { /// Input execution plan input: Arc, /// Partitioning scheme to use - partitioning: Partitioning, + partitioning: CometPartitioning, /// Output data file path output_data_file: String, /// Output index file path @@ -84,7 +86,7 @@ impl ShuffleWriterExec { /// Create a new ShuffleWriterExec pub fn try_new( input: Arc, - partitioning: Partitioning, + partitioning: CometPartitioning, codec: CompressionCodec, output_data_file: String, output_index_file: String, @@ -92,7 +94,7 @@ impl ShuffleWriterExec { ) -> Result { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&input.schema())), - partitioning.clone(), + Partitioning::UnknownPartitioning(1), EmissionType::Final, Boundedness::Bounded, ); @@ -111,7 +113,7 @@ impl ShuffleWriterExec { } impl DisplayAs for ShuffleWriterExec { - fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { write!( @@ -209,7 +211,7 @@ async fn external_shuffle( partition: usize, output_data_file: String, output_index_file: String, - partitioning: Partitioning, + partitioning: CometPartitioning, metrics: ShuffleRepartitionerMetrics, context: Arc, codec: CompressionCodec, @@ -321,7 +323,7 @@ struct MultiPartitionShuffleRepartitioner { partition_writers: Vec, shuffle_block_writer: ShuffleBlockWriter, /// Partitioning scheme to use - partitioning: Partitioning, + partitioning: CometPartitioning, runtime: Arc, metrics: ShuffleRepartitionerMetrics, /// Reused scratch space for computing partition indices @@ -331,6 +333,10 @@ struct MultiPartitionShuffleRepartitioner { /// Reservation for repartitioning reservation: MemoryReservation, tracing_enabled: bool, + /// RangePartitioning-specific state + bounds_rows: Option>, + row_converter: Option, + seed: u64, } #[derive(Default)] @@ -356,7 +362,7 @@ impl MultiPartitionShuffleRepartitioner { output_data_file: String, output_index_file: String, schema: SchemaRef, - partitioning: Partitioning, + partitioning: CometPartitioning, metrics: ShuffleRepartitionerMetrics, runtime: Arc, batch_size: usize, @@ -373,7 +379,11 @@ impl MultiPartitionShuffleRepartitioner { // initialization code is simply initializing the vectors to the desired size. // The initial values are not used. let scratch = ScratchSpace { - hashes_buf: vec![0; batch_size], + hashes_buf: match partitioning { + // Only allocate the hashes_buf if hash partitioning. + CometPartitioning::Hash(_, _) => vec![0; batch_size], + _ => vec![], + }, partition_ids: vec![0; batch_size], partition_row_indices: vec![0; batch_size], partition_starts: vec![0; num_output_partitions + 1], @@ -403,6 +413,10 @@ impl MultiPartitionShuffleRepartitioner { batch_size, reservation, tracing_enabled, + bounds_rows: None, + row_converter: None, + // Spark RangePartitioner seeds off of partition number. + seed: partition as u64, }) } @@ -416,6 +430,52 @@ impl MultiPartitionShuffleRepartitioner { return Ok(()); } + fn map_partition_ids_to_starts_and_indices( + scratch: &mut ScratchSpace, + num_output_partitions: usize, + num_rows: usize, + ) { + let partition_ids = &mut scratch.partition_ids[..num_rows]; + + // count each partition size, while leaving the last extra element as 0 + let partition_counters = &mut scratch.partition_starts; + partition_counters.resize(num_output_partitions + 1, 0); + partition_counters.fill(0); + partition_ids + .iter() + .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); + + // accumulate partition counters into partition ends + // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] + let partition_ends = partition_counters; + let mut accum = 0; + partition_ends.iter_mut().for_each(|v| { + *v += accum; + accum = *v; + }); + + // calculate partition row indices and partition starts + // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] will produce the following partition_row_indices + // and partition_starts arrays: + // + // partition_row_indices: [6, 1, 2, 3, 4, 5, 0] + // partition_starts: [0, 1, 4, 6, 7] + // + // partition_starts conceptually splits partition_row_indices into smaller slices. + // Each slice partition_row_indices[partition_starts[K]..partition_starts[K + 1]] contains the + // row indices of the input batch that are partitioned into partition K. For example, + // first partition 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. + let partition_row_indices = &mut scratch.partition_row_indices; + partition_row_indices.resize(num_rows, 0); + for (index, partition_id) in partition_ids.iter().enumerate().rev() { + partition_ends[*partition_id as usize] -= 1; + let end = partition_ends[*partition_id as usize]; + partition_row_indices[end as usize] = index as u32; + } + + // after calculating, partition ends become partition starts + } + if input.num_rows() > self.batch_size { return Err(DataFusionError::Internal( "Input batch size exceeds configured batch size. Call `insert_batch` instead." @@ -431,72 +491,119 @@ impl MultiPartitionShuffleRepartitioner { self.metrics.baseline.record_output(input.num_rows()); match &self.partitioning { - Partitioning::Hash(exprs, num_output_partitions) => { + CometPartitioning::Hash(exprs, num_output_partitions) => { let mut scratch = std::mem::take(&mut self.scratch); let (partition_starts, partition_row_indices): (&Vec, &Vec) = { let mut timer = self.metrics.repart_time.timer(); - // evaluate partition expressions + // Evaluate partition expressions to get rows to apply partitioning scheme. let arrays = exprs .iter() .map(|expr| expr.evaluate(&input)?.into_array(input.num_rows())) .collect::>>()?; - // use identical seed as spark hash partition - let hashes_buf = &mut scratch.hashes_buf[..arrays[0].len()]; + let num_rows = arrays[0].len(); + + // Use identical seed as Spark hash partitioning. + let hashes_buf = &mut scratch.hashes_buf[..num_rows]; hashes_buf.fill(42_u32); - // Hash arrays and compute buckets based on number of partitions - let partition_ids = &mut scratch.partition_ids[..arrays[0].len()]; - create_murmur3_hashes(&arrays, hashes_buf)? - .iter() - .enumerate() - .for_each(|(idx, hash)| { - partition_ids[idx] = pmod(*hash, *num_output_partitions) as u32; - }); - - // count each partition size, while leaving the last extra element as 0 - let partition_counters = &mut scratch.partition_starts; - partition_counters.resize(num_output_partitions + 1, 0); - partition_counters.fill(0); - partition_ids + // Generate partition ids for every row. + { + // Hash arrays and compute partition ids based on number of partitions. + let partition_ids = &mut scratch.partition_ids[..num_rows]; + create_murmur3_hashes(&arrays, hashes_buf)? + .iter() + .enumerate() + .for_each(|(idx, hash)| { + partition_ids[idx] = pmod(*hash, *num_output_partitions) as u32; + }); + } + + // We now have partition ids for every input row, map that to partition starts + // and partition indices to eventually right these rows to partition buffers. + map_partition_ids_to_starts_and_indices( + &mut scratch, + *num_output_partitions, + num_rows, + ); + + timer.stop(); + Ok::<(&Vec, &Vec), DataFusionError>(( + &scratch.partition_starts, + &scratch.partition_row_indices, + )) + }?; + + self.buffer_partitioned_batch_may_spill( + input, + partition_row_indices, + partition_starts, + ) + .await?; + self.scratch = scratch; + } + CometPartitioning::RangePartitioning( + lex_ordering, + num_output_partitions, + sample_size, + ) => { + let mut scratch = std::mem::take(&mut self.scratch); + let (partition_starts, partition_row_indices): (&Vec, &Vec) = { + let mut timer = self.metrics.repart_time.timer(); + + // Evaluate partition expressions for values to apply partitioning scheme on. + let arrays = lex_ordering .iter() - .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); - - // accumulate partition counters into partition ends - // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] - let partition_ends = partition_counters; - let mut accum = 0; - partition_ends.iter_mut().for_each(|v| { - *v += accum; - accum = *v; - }); - - // calculate partition row indices and partition starts - // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] will produce the following partition_row_indices - // and partition_starts arrays: - // - // partition_row_indices: [6, 1, 2, 3, 4, 5, 0] - // partition_starts: [0, 1, 4, 6, 7] - // - // partition_starts conceptually splits partition_row_indices into smaller slices. - // Each slice partition_row_indices[partition_starts[K]..partition_starts[K + 1]] contains the - // row indices of the input batch that are partitioned into partition K. For example, - // first partition 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. - let partition_row_indices = &mut scratch.partition_row_indices; - partition_row_indices.resize(input.num_rows(), 0); - for (index, partition_id) in partition_ids.iter().enumerate().rev() { - partition_ends[*partition_id as usize] -= 1; - let end = partition_ends[*partition_id as usize]; - partition_row_indices[end as usize] = index as u32; + .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) + .collect::>>()?; + + let num_rows = arrays[0].len(); + + // If necessary (i.e., when first batch arrives) generate the bounds (as Rows) + // for range partitioning based on randomly reservoir sampling the batch. + if self.row_converter.is_none() { + let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( + &arrays, + lex_ordering, + *num_output_partitions, + input.num_rows(), + *sample_size, + self.seed, + )?; + + self.bounds_rows = + Some(bounds_rows.iter().map(|row| row.owned()).collect_vec()); + self.row_converter = Some(row_converter); } - // after calculating, partition ends become partition starts - let partition_starts = partition_ends; + // Generate partition ids for every row, first by converting the partition + // arrays to Rows, and then doing binary search for each Row against the + // bounds Rows. + let row_batch = self + .row_converter + .as_ref() + .unwrap() + .convert_columns(arrays.as_slice())?; + + RangePartitioner::partition_indices_for_batch( + &row_batch, + self.bounds_rows.as_ref().unwrap().as_slice(), + &mut scratch.partition_ids[..num_rows], + ); + + // We now have partition ids for every input row, map that to partition starts + // and partition indices to eventually right these rows to partition buffers. + map_partition_ids_to_starts_and_indices( + &mut scratch, + *num_output_partitions, + num_rows, + ); + timer.stop(); Ok::<(&Vec, &Vec), DataFusionError>(( - partition_starts, - partition_row_indices, + &scratch.partition_starts, + &scratch.partition_row_indices, )) }?; @@ -512,7 +619,7 @@ impl MultiPartitionShuffleRepartitioner { // this should be unreachable as long as the validation logic // in the constructor is kept up-to-date return Err(DataFusionError::NotImplemented(format!( - "Unsupported repartitioning scheme {:?}", + "Unsupported shuffle partitioning scheme {:?}", other ))); } @@ -1175,7 +1282,8 @@ mod test { use datafusion::datasource::source::DataSourceExec; use datafusion::execution::config::SessionConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; - use datafusion::physical_expr::expressions::Column; + use datafusion::physical_expr::expressions::{col, Column}; + use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion::physical_plan::common::collect; use datafusion::prelude::SessionContext; use std::io::Cursor; @@ -1253,7 +1361,7 @@ mod test { "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), batch.schema(), - Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), + CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), ShuffleRepartitionerMetrics::new(&metrics_set, 0), runtime_env, 1024, @@ -1296,34 +1404,45 @@ mod test { ) { let batch = create_batch(batch_size); - let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); - - let partitions = &[batches]; - let exec = ShuffleWriterExec::try_new( - Arc::new(DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new(partitions, batch.schema(), None).unwrap(), - ))), - Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), - CompressionCodec::Zstd(1), - "/tmp/data.out".to_string(), - "/tmp/index.out".to_string(), - false, - ) - .unwrap(); + for partitioning in [ + CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), + CometPartitioning::RangePartitioning( + LexOrdering::new(vec![PhysicalSortExpr::new_default( + col("a", batch.schema().as_ref()).unwrap(), + )]), + num_partitions, + 100, + ), + ] { + let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); - // 10MB memory should be enough for running this test - let config = SessionConfig::new(); - let mut runtime_env_builder = RuntimeEnvBuilder::new(); - runtime_env_builder = match memory_limit { - Some(limit) => runtime_env_builder.with_memory_limit(limit, 1.0), - None => runtime_env_builder, - }; - let runtime_env = Arc::new(runtime_env_builder.build().unwrap()); - let ctx = SessionContext::new_with_config_rt(config, runtime_env); - let task_ctx = ctx.task_ctx(); - let stream = exec.execute(0, task_ctx).unwrap(); - let rt = Runtime::new().unwrap(); - rt.block_on(collect(stream)).unwrap(); + let partitions = &[batches]; + let exec = ShuffleWriterExec::try_new( + Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(partitions, batch.schema(), None).unwrap(), + ))), + partitioning, + CompressionCodec::Zstd(1), + "/tmp/data.out".to_string(), + "/tmp/index.out".to_string(), + false, + ) + .unwrap(); + + // 10MB memory should be enough for running this test + let config = SessionConfig::new(); + let mut runtime_env_builder = RuntimeEnvBuilder::new(); + runtime_env_builder = match memory_limit { + Some(limit) => runtime_env_builder.with_memory_limit(limit, 1.0), + None => runtime_env_builder, + }; + let runtime_env = Arc::new(runtime_env_builder.build().unwrap()); + let ctx = SessionContext::new_with_config_rt(config, runtime_env); + let task_ctx = ctx.task_ctx(); + let stream = exec.execute(0, task_ctx).unwrap(); + let rt = Runtime::new().unwrap(); + rt.block_on(collect(stream)).unwrap(); + } } fn create_batch(batch_size: usize) -> RecordBatch { diff --git a/native/proto/src/proto/partitioning.proto b/native/proto/src/proto/partitioning.proto index 21bd505609..ea0e586fb1 100644 --- a/native/proto/src/proto/partitioning.proto +++ b/native/proto/src/proto/partitioning.proto @@ -28,15 +28,22 @@ option java_package = "org.apache.comet.serde"; // The basic message representing a Spark partitioning. message Partitioning { oneof partitioning_struct { - HashRepartition hash_partition = 2; - SinglePartition single_partition = 3; + HashPartition hash_partition = 1; + SinglePartition single_partition = 2; + RangePartition range_partition = 3; } } -message HashRepartition { +message HashPartition { repeated spark.spark_expression.Expr hash_expression = 1; int32 num_partitions = 2; } message SinglePartition { } + +message RangePartition { + repeated spark.spark_expression.Expr sort_orders = 1; + int32 num_partitions = 2; + int32 sample_size = 3; +} diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 13bea457d3..9c988bdae3 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -39,8 +39,7 @@ import org.apache.spark.sql.execution import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec} -import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD} -import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile} import org.apache.spark.sql.execution.datasources.v2.{DataSourceRDD, DataSourceRDDPartition} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashJoin, ShuffledHashJoinExec, SortMergeJoinExec} @@ -2878,7 +2877,7 @@ object QueryPlanSerde extends Logging with CometExprShim { * Hash Partition Key determines how data should be collocated for operations like * `groupByKey`, `reduceByKey` or `join`. */ - def supportedPartitionKeyDataType(dt: DataType): Boolean = dt match { + def supportedHashPartitionKeyDataType(dt: DataType): Boolean = dt match { case _: BooleanType | _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType | _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: TimestampNTZType | _: DecimalType | _: DateType => @@ -2889,6 +2888,7 @@ object QueryPlanSerde extends Logging with CometExprShim { val inputs = s.child.output val partitioning = s.outputPartitioning + val conf = SQLConf.get var msg = "" val supported = partitioning match { case HashPartitioning(expressions, _) => @@ -2896,14 +2896,23 @@ object QueryPlanSerde extends Logging with CometExprShim { // due to lack of hashing support for those types val supported = expressions.map(QueryPlanSerde.exprToProto(_, inputs)).forall(_.isDefined) && - expressions.forall(e => supportedPartitionKeyDataType(e.dataType)) && - inputs.forall(attr => supportedShuffleDataType(attr.dataType)) + expressions.forall(e => supportedHashPartitionKeyDataType(e.dataType)) && + inputs.forall(attr => supportedShuffleDataType(attr.dataType)) && + CometConf.COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED.get(conf) if (!supported) { - msg = s"unsupported Spark partitioning expressions: $expressions" + msg = s"unsupported Spark partitioning: $expressions" } supported case SinglePartition => inputs.forall(attr => supportedShuffleDataType(attr.dataType)) + case RangePartitioning(ordering, _) => + val supported = ordering.map(QueryPlanSerde.exprToProto(_, inputs)).forall(_.isDefined) && + inputs.forall(attr => supportedShuffleDataType(attr.dataType)) && + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.get(conf) + if (!supported) { + msg = s"unsupported Spark partitioning: $ordering" + } + supported case _ => msg = s"unsupported Spark partitioning: ${partitioning.getClass.getName}" false diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 66d2fac89c..5d772be403 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -25,12 +25,14 @@ import java.nio.file.{Files, Paths} import scala.collection.JavaConverters.asJavaIterableConverter import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriteMetricsReporter, ShuffleWriter} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, SinglePartition} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, SinglePartition} import org.apache.spark.sql.comet.{CometExec, CometMetricNode} import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.comet.CometConf @@ -50,7 +52,8 @@ class CometNativeShuffleWriter[K, V]( mapId: Long, context: TaskContext, metricsReporter: ShuffleWriteMetricsReporter) - extends ShuffleWriter[K, V] { + extends ShuffleWriter[K, V] + with Logging { private val OFFSET_LENGTH = 8 @@ -170,7 +173,7 @@ class CometNativeShuffleWriter[K, V]( case _: HashPartitioning => val hashPartitioning = outputPartitioning.asInstanceOf[HashPartitioning] - val partitioning = PartitioningOuterClass.HashRepartition.newBuilder() + val partitioning = PartitioningOuterClass.HashPartition.newBuilder() partitioning.setNumPartitions(outputPartitioning.numPartitions) val partitionExprs = hashPartitioning.expressions @@ -186,7 +189,44 @@ class CometNativeShuffleWriter[K, V]( val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() shuffleWriterBuilder.setPartitioning( partitioningBuilder.setHashPartition(partitioning).build()) + case _: RangePartitioning => + val rangePartitioning = outputPartitioning.asInstanceOf[RangePartitioning] + val partitioning = PartitioningOuterClass.RangePartition.newBuilder() + partitioning.setNumPartitions(outputPartitioning.numPartitions) + val sampleSize = { + // taken from org.apache.spark.RangePartitioner#rangeBounds + // This is the sample size we need to have roughly balanced output partitions, + // capped at 1M. + // Cast to double to avoid overflowing ints or longs + val sampleSize = math.min( + SQLConf.get + .getConf(SQLConf.RANGE_EXCHANGE_SAMPLE_SIZE_PER_PARTITION) + .toDouble * outputPartitioning.numPartitions, + 1e6) + // Assume the input partitions are roughly balanced and over-sample a little bit. + // Comet: we don't divide by numPartitions since each DF plan handles one partition. + math.ceil(3.0 * sampleSize).toInt + } + if (sampleSize > 8192) { + logWarning( + s"RangePartitioning sampleSize of s$sampleSize exceeds Comet RecordBatch size.") + } + partitioning.setSampleSize(sampleSize) + + val orderingExprs = rangePartitioning.ordering + .flatMap(e => QueryPlanSerde.exprToProto(e, outputAttributes)) + + if (orderingExprs.length != rangePartitioning.ordering.length) { + throw new UnsupportedOperationException( + s"Partitioning $rangePartitioning is not supported.") + } + + partitioning.addAllSortOrders(orderingExprs.asJava) + + val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() + shuffleWriterBuilder.setPartitioning( + partitioningBuilder.setRangePartition(partitioning).build()) case SinglePartition => val partitioning = PartitioningOuterClass.SinglePartition.newBuilder() diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt index 5ff3a2a9ba..61c6e2f564 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (34) +- CometSort (33) - +- CometColumnarExchange (32) + +- CometExchange (32) +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometProject (20) @@ -180,9 +180,9 @@ Arguments: [d_week_seq1#29], [(d_week_seq2#54 - 53)], Inner, BuildRight Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#55), 2) AS round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1#31 / mon_sales2#56), 2) AS round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1#32 / tue_sales2#57), 2) AS round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1#33 / wed_sales2#58), 2) AS round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1#34 / thu_sales2#59), 2) AS round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1#35 / fri_sales2#60), 2) AS round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1#36 / sat_sales2#61), 2) AS round((sat_sales1 / sat_sales2), 2)#68] -(32) CometColumnarExchange +(32) CometExchange Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (33) CometSort Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt index 8e6713bf26..a38ccedce7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [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)] - CometColumnarExchange [d_week_seq1] #1 + CometExchange [d_week_seq1] #1 CometProject [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] [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)] CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt index 6ce03737eb..2a4b0fc580 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (90) +- CometSort (89) - +- CometColumnarExchange (88) + +- CometExchange (88) +- CometProject (87) +- CometBroadcastHashJoin (86) :- CometProject (73) @@ -496,9 +496,9 @@ Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.0 Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] -(88) CometColumnarExchange +(88) CometExchange Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] (89) CometSort Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt index 22635bb9f2..4591720269 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometColumnarExchange [ca_county] #1 + CometExchange [ca_county] #1 CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt index 13934b6c66..ce3305fae3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt index 7f1cb740c4..9bc47a48a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt index 0b404d5868..ec8969bbd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt index 9df61b1669..8397df62dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt index 683ec549cc..cf2ac9b891 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt index 9df61b1669..8397df62dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt index 03ba8783bb..46e0ae9610 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt index ffcec55ac5..137ee8251e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [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] - CometColumnarExchange [product_name,store_name,cnt] #1 + CometExchange [product_name,store_name,cnt] #1 CometProject [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] CometSortMergeJoin [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,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt index 9539905b2c..b596f48235 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (37) +- CometSort (36) - +- CometColumnarExchange (35) + +- CometExchange (35) +- CometHashAggregate (34) +- CometExchange (33) +- CometHashAggregate (32) @@ -201,9 +201,9 @@ Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] Functions [1]: [sum(UnscaledValue(ext_price#14))] -(35) CometColumnarExchange +(35) CometExchange Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (36) CometSort Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt index d765255d14..49adad0a02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 + CometExchange [ext_price,brand_id] #1 CometHashAggregate [brand_id,brand,t_hour,t_minute,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt index 77957c5099..87ceb7a009 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt index e3c1c0b82e..92802523fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 + CometExchange [cnt] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt index b5f0f9083b..dc3a05dbbc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (43) +- CometSort (42) - +- CometColumnarExchange (41) + +- CometExchange (41) +- CometHashAggregate (40) +- CometExchange (39) +- CometHashAggregate (38) @@ -234,9 +234,9 @@ Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -(41) CometColumnarExchange +(41) CometExchange Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (42) CometSort Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt index 3b302272e0..c160227efd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 + CometExchange [Returns_Loss] #1 CometHashAggregate [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt index 13934b6c66..ce3305fae3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt index 7f1cb740c4..9bc47a48a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt index 720daf075d..e3c22a1c5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (186) +- CometSort (185) - +- CometColumnarExchange (184) + +- CometExchange (184) +- CometProject (183) +- CometSortMergeJoin (182) :- CometSort (113) @@ -1019,9 +1019,9 @@ Arguments: [item_sk#95, store_name#96, store_zip#97], [item_sk#181, store_name#1 Input [25]: [product_name#94, item_sk#95, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, item_sk#181, store_name#182, store_zip#183, syear#184, cnt#185, s1#186, s2#187, s3#188] Arguments: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185], [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] -(184) CometColumnarExchange +(184) CometExchange Input [21]: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] -Arguments: rangepartitioning(product_name#94 ASC NULLS FIRST, store_name#96 ASC NULLS FIRST, cnt#185 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +Arguments: rangepartitioning(product_name#94 ASC NULLS FIRST, store_name#96 ASC NULLS FIRST, cnt#185 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] (185) CometSort Input [21]: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt index 043b82dfa2..35f71c1207 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (39) CometColumnarToRow InputAdapter CometSort [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] - CometColumnarExchange [product_name,store_name,cnt] #1 + CometExchange [product_name,store_name,cnt] #1 CometProject [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] CometSortMergeJoin [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,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt index 77957c5099..87ceb7a009 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt index e3c1c0b82e..92802523fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 + CometExchange [cnt] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index c83c8be0a3..5b4a6372c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (34) +- CometSort (33) - +- CometColumnarExchange (32) + +- CometExchange (32) +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometProject (20) @@ -180,9 +180,9 @@ Arguments: [d_week_seq1#29], [(d_week_seq2#46 - 53)], Inner, BuildRight Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#47), 2) AS round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1#31 / mon_sales2#48), 2) AS round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1#32 / tue_sales2#49), 2) AS round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1#33 / wed_sales2#50), 2) AS round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1#34 / thu_sales2#51), 2) AS round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1#35 / fri_sales2#52), 2) AS round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1#36 / sat_sales2#53), 2) AS round((sat_sales1 / sat_sales2), 2)#60] -(32) CometColumnarExchange +(32) CometExchange Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (33) CometSort Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index 8e6713bf26..a38ccedce7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [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)] - CometColumnarExchange [d_week_seq1] #1 + CometExchange [d_week_seq1] #1 CometProject [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] [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)] CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index 6ce03737eb..2a4b0fc580 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (90) +- CometSort (89) - +- CometColumnarExchange (88) + +- CometExchange (88) +- CometProject (87) +- CometBroadcastHashJoin (86) :- CometProject (73) @@ -496,9 +496,9 @@ Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.0 Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] -(88) CometColumnarExchange +(88) CometExchange Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] (89) CometSort Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index 22635bb9f2..4591720269 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometColumnarExchange [ca_county] #1 + CometExchange [ca_county] #1 CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index 13934b6c66..ce3305fae3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index 7f1cb740c4..9bc47a48a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index 0b404d5868..ec8969bbd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 9df61b1669..8397df62dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 683ec549cc..cf2ac9b891 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 9df61b1669..8397df62dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index 03ba8783bb..46e0ae9610 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index ffcec55ac5..137ee8251e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [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] - CometColumnarExchange [product_name,store_name,cnt] #1 + CometExchange [product_name,store_name,cnt] #1 CometProject [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] CometSortMergeJoin [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,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 9539905b2c..b596f48235 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (37) +- CometSort (36) - +- CometColumnarExchange (35) + +- CometExchange (35) +- CometHashAggregate (34) +- CometExchange (33) +- CometHashAggregate (32) @@ -201,9 +201,9 @@ Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] Functions [1]: [sum(UnscaledValue(ext_price#14))] -(35) CometColumnarExchange +(35) CometExchange Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (36) CometSort Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index d765255d14..49adad0a02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 + CometExchange [ext_price,brand_id] #1 CometHashAggregate [brand_id,brand,t_hour,t_minute,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index 77957c5099..87ceb7a009 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index e3c1c0b82e..92802523fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 + CometExchange [cnt] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index b5f0f9083b..dc3a05dbbc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (43) +- CometSort (42) - +- CometColumnarExchange (41) + +- CometExchange (41) +- CometHashAggregate (40) +- CometExchange (39) +- CometHashAggregate (38) @@ -234,9 +234,9 @@ Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -(41) CometColumnarExchange +(41) CometExchange Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (42) CometSort Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index 3b302272e0..c160227efd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 + CometExchange [Returns_Loss] #1 CometHashAggregate [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt index 37fa3cf15f..1e73653868 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt index 7bef10cb97..d0b65bd9cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt index 2e195d003a..2d4d9b9dc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt index e3c8d3cd2c..9c6b8f1985 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [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] - CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + CometExchange [product_name,store_name,cnt,s1,s1] #1 CometProject [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] CometSortMergeJoin [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,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt index 37fa3cf15f..1e73653868 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt index 7bef10cb97..d0b65bd9cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt index c5e3046383..f27e04a898 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (186) +- CometSort (185) - +- CometColumnarExchange (184) + +- CometExchange (184) +- CometProject (183) +- CometSortMergeJoin (182) :- CometSort (113) @@ -1019,9 +1019,9 @@ Arguments: [item_sk#95, store_name#96, store_zip#97], [item_sk#181, store_name#1 Input [25]: [product_name#94, item_sk#95, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, item_sk#181, store_name#182, store_zip#183, syear#184, cnt#185, s1#186, s2#187, s3#188] Arguments: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185], [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] -(184) CometColumnarExchange +(184) CometExchange Input [21]: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] -Arguments: rangepartitioning(product_name#94 ASC NULLS FIRST, store_name#96 ASC NULLS FIRST, cnt#185 ASC NULLS FIRST, s1#108 ASC NULLS FIRST, s1#186 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +Arguments: rangepartitioning(product_name#94 ASC NULLS FIRST, store_name#96 ASC NULLS FIRST, cnt#185 ASC NULLS FIRST, s1#108 ASC NULLS FIRST, s1#186 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] (185) CometSort Input [21]: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt index 356584ffa5..4e32f307ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (39) CometColumnarToRow InputAdapter CometSort [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] - CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + CometExchange [product_name,store_name,cnt,s1,s1] #1 CometProject [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] CometSortMergeJoin [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,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt index 37fa3cf15f..1e73653868 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 7bef10cb97..d0b65bd9cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 2e195d003a..2d4d9b9dc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index e3c8d3cd2c..9c6b8f1985 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [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] - CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + CometExchange [product_name,store_name,cnt,s1,s1] #1 CometProject [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] CometSortMergeJoin [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,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [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] diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala index a1b1812b31..433bdf4675 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala @@ -162,7 +162,9 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { // cannot run fully natively due to range partitioning and sort val (_, cometPlan) = checkSparkAnswer(sql) if (usingDataSourceExec) { + // Native scans and native shuffle should support all data types for this query assert(1 == collectNativeScans(cometPlan).length) + assert(1 == collectCometShuffleExchanges(cometPlan).length) } } } @@ -187,7 +189,9 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { // cannot run fully natively due to range partitioning and sort val (_, cometPlan) = checkSparkAnswer(sql) if (usingDataSourceExec) { + // Native scans and native shuffle should support all data types for this query assert(1 == collectNativeScans(cometPlan).length) + assert(1 == collectCometShuffleExchanges(cometPlan).length) } } @@ -250,10 +254,31 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { val df = spark.read.parquet(filename) val df2 = df.repartition(8, df.col("c0")).sort("c1") df2.collect() - if (usingDataSourceExec) { - val cometShuffles = collectCometShuffleExchanges(df2.queryExecution.executedPlan) - assert(1 == cometShuffles.length) + val expectedNumCometShuffles = CometConf.COMET_NATIVE_SCAN_IMPL.get() match { + case CometConf.SCAN_NATIVE_COMET => + CometConf.COMET_SHUFFLE_MODE.get() match { + case "jvm" => + // Uses a single CometColumnarExchange for hash partitioning + // Range partitioning is in Spark + 1 + case "native" => + // Uses Spark for hash partitioning and range partitioning + 0 + } + case CometConf.SCAN_NATIVE_ICEBERG_COMPAT | CometConf.SCAN_NATIVE_DATAFUSION => + CometConf.COMET_SHUFFLE_MODE.get() match { + case "jvm" => + // Uses a single CometColumnarExchange for hash partitioning + // Range partitioning is in Spark + 1 + case "native" => + // Uses Comet for hash partitioning and range partitioning + 2 + } } + + val cometShuffles = collectCometShuffleExchanges(df2.queryExecution.executedPlan) + assert(expectedNumCometShuffles == cometShuffles.length) } test("join") { diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index e1cbd7406a..4543f32913 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -121,28 +121,28 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper } test("native operator after native shuffle") { - withParquetTable((0 until 5).map(i => (i, (i + 1).toLong)), "tbl") { - val df = sql("SELECT * FROM tbl") - - val shuffled1 = df - .repartition(10, $"_2") - .select($"_1", $"_1" + 1, $"_2" + 2) - .repartition(10, $"_1") - .filter($"_1" > 1) - - // 2 Comet shuffle exchanges are expected - checkShuffleAnswer(shuffled1, 2) - - val shuffled2 = df - .repartitionByRange(10, $"_2") - .select($"_1", $"_1" + 1, $"_2" + 2) - .repartition(10, $"_1") - .filter($"_1" > 1) - - // Because the first exchange from the bottom is range exchange which native shuffle - // doesn't support. So Comet exec operators stop before the first exchange and thus - // there is no Comet exchange. - checkShuffleAnswer(shuffled2, 0) + Seq("true", "false").zip(Seq("true", "false")).foreach { partitioning => + withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED.key -> partitioning._1, + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> partitioning._2) { + withParquetTable((0 until 5).map(i => (i, (i + 1).toLong)), "tbl") { + val df = sql("SELECT * FROM tbl") + + val shuffled = df + .repartition(10, $"_2") + .select($"_1", $"_1" + 1, $"_2" + 2) + .repartition(10, $"_1") + .filter($"_1" > 1) + + // We expect a hash and range partitioned exchanges. If both are true, we'll get two + // native exchanges. Otherwise both will fall back. + if (partitioning._1 == "true" && partitioning._2 == "true") { + checkShuffleAnswer(shuffled, 2) + } else { + checkShuffleAnswer(shuffled, 0) + } + } + } } }